You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by hvanhovell <gi...@git.apache.org> on 2016/03/07 10:40:43 UTC

[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

GitHub user hvanhovell opened a pull request:

    https://github.com/apache/spark/pull/11557

    [SPARK-13713][SQL] Migrate parser from ANTLR3 to ANTLR4 [WIP]

    ### What changes were proposed in this pull request?
    The current ANTLR3 parser is quite complex to maintain and suffers from code blow-ups. This PR introduces a new parser that is based on ANTLR4.
    
    This parser is based on the [Presto's SQL parser](https://github.com/facebook/presto/blob/master/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4). The current implementation can parse and create Catalyst and SQL plans. Large parts of the HiveQl DDL and some of the DML functionality is currently missing, the plan is to add this in follow-up PRs.
    
    This PR is a work in progress, and work needs to be done in the following area's:
    
    - [ ] Error handling should be improved.
    - [ ] Documentation should be improved.
    - [ ] Multi-Insert needs to be tested.
    
    
    ### How was this patch tested?
    
    Catalyst and SQL unit tests.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/hvanhovell/spark ngParser

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/11557.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #11557
    
----
commit 567032345603f445bbdea14a9f17c638119d93d6
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-02-23T16:30:10Z

    Move Presto parser into spark

commit acac8f5233dcea16558c97853315a8f0ff677c74
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-02-23T22:21:13Z

    Build parsers side-by-side

commit 1a98c0ccf6c9200a837d3800118128c94590de38
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-02-25T23:26:23Z

    Add Expression Parsing

commit 3ebdeecc23a9eb63b7d1f4272b6bf43e234f9722
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-02-27T17:10:56Z

    Add Expression Parsing

commit 555a959b7ac333a6f485b0097e532f122cb179e9
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-02-29T23:49:40Z

    Add Basic Plan Parsing

commit a130429953713fbec3408e4b0e8db64ae4fa5c1c
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-06T00:25:26Z

    Catalyst feature parity. Start with SQL/Core.

commit 384f10a4c106200ffc3dae9e0e60e8f3c15a2ee5
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-06T21:28:21Z

    SQL/core feature parity.

commit d4461ae82316f16a5408eda5773be3502ea8d25d
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-06T21:34:21Z

    Merge remote-tracking branch 'apache-github/master' into ngParser
    
    # Conflicts:
    #	sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala

commit e991d20e2949ad25c7a26f15567105a52e5b0196
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-06T21:34:34Z

    Merge remote-tracking branch 'apache-github/master' into ngParser
    
    # Conflicts:
    #	sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala

commit 8ff792988d4b91fdfa0978c65368e560d79266ab
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-06T23:24:16Z

    Make tests pass.

commit 06303bcc2801b3f7520c3174a982e21056e86386
Author: Herman van Hovell <hv...@questtec.nl>
Date:   2016-03-07T06:52:36Z

    Fix expand star

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57354160
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    --- End diff --
    
    Ah... now I see. HiveQl allows us to have function dot separated function names. So I am just making sure this works. I am not sure how much sense this makes further down the line.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #11557: [SPARK-13713][SQL] Migrate parser from ANTLR3 to ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r73677314
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1452 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import ParserUtils._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
    +    visit(ctx.dataType).asInstanceOf[DataType]
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Make sure we do not try to create a plan for a native command.
    +   */
    +  override def visitExecuteNativeCommand(ctx: ExecuteNativeCommandContext): LogicalPlan = null
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          throw new ParseException("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(string(pattern)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }
    +
    +      // Check for duplicate names.
    +      ctes.groupBy(_._1).filter(_._2.size > 1).foreach {
    +        case (name, _) =>
    +          throw new ParseException(
    +            s"Name '$name' is used for multiple common table expressions", ctx)
    +      }
    +
    +      With(query, ctes.toMap)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryResultClauses).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryResultClauses).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Create a partition specification map.
    +   */
    +  override def visitPartitionSpec(
    +      ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) {
    +    ctx.partitionVal.asScala.map { pVal =>
    +      val name = pVal.identifier.getText.toLowerCase
    +      val value = Option(pVal.constant).map(visitStringConstant)
    +      name -> value
    +    }.toMap
    +  }
    +
    +  /**
    +   * Create a partition specification map without optional values.
    +   */
    +  protected def visitNonOptionalPartitionSpec(
    +      ctx: PartitionSpecContext): Map[String, String] = withOrigin(ctx) {
    +    visitPartitionSpec(ctx).mapValues(_.orNull).map(identity)
    +  }
    +
    +  /**
    +   * Convert a constant of any type into a string. This is typically used in DDL commands, and its
    +   * main purpose is to prevent slight differences due to back to back conversions i.e.:
    +   * String -> Literal -> String.
    +   */
    +  protected def visitStringConstant(ctx: ConstantContext): String = withOrigin(ctx) {
    +    ctx match {
    +      case s: StringLiteralContext => createString(s)
    +      case o => o.getText
    +    }
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan. These
    +   * clauses determine the shape (ordering/partitioning/rows) of the query result.
    +   */
    +  private def withQueryResultClauses(
    +      ctx: QueryOrganizationContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
    +    val withOrder = if (
    +      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // ORDER BY ...
    +      Sort(order.asScala.map(visitSortItem), global = true, query)
    +    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ...
    +      Sort(sort.asScala.map(visitSortItem), global = false, query)
    +    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // DISTRIBUTE BY ...
    +      RepartitionByExpression(expressionList(distributeBy), query)
    +    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ... DISTRIBUTE BY ...
    +      Sort(
    +        sort.asScala.map(visitSortItem),
    +        global = false,
    +        RepartitionByExpression(expressionList(distributeBy), query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
    +      // CLUSTER BY ...
    +      val expressions = expressionList(clusterBy)
    +      Sort(
    +        expressions.map(SortOrder(_, Ascending)),
    +        global = false,
    +        RepartitionByExpression(expressions, query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // [EMPTY]
    +      query
    +    } else {
    +      throw new ParseException(
    +        "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
    +    }
    +
    +    // WINDOWS
    +    val withWindow = withOrder.optionalMap(windows)(withWindows)
    +
    +    // LIMIT
    +    withWindow.optional(limit) {
    +      Limit(typedVisit(limit), withWindow)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan using a query specification.
    +   */
    +  override def visitQuerySpecification(
    +      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
    +    val from = OneRowRelation.optional(ctx.fromClause) {
    +      visitFromClause(ctx.fromClause)
    +    }
    +    withQuerySpecification(ctx, from)
    +  }
    +
    +  /**
    +   * Add a query specification to a logical plan. The query specification is the core of the logical
    +   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
    +   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
    +   *
    +   * Note that query hints are ignored (both by the parser and the builder).
    +   */
    +  private def withQuerySpecification(
    +      ctx: QuerySpecificationContext,
    +      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // WHERE
    +    def filter(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = {
    +      Filter(expression(ctx), plan)
    +    }
    +
    +    // Expressions.
    +    val expressions = Option(namedExpressionSeq).toSeq
    +      .flatMap(_.namedExpression.asScala)
    +      .map(typedVisit[Expression])
    +
    +    // Create either a transform or a regular query.
    +    val specType = Option(kind).map(_.getType).getOrElse(SqlBaseParser.SELECT)
    +    specType match {
    +      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
    +        // Transform
    +
    +        // Add where.
    +        val withFilter = relation.optionalMap(where)(filter)
    +
    +        // Create the attributes.
    +        val (attributes, schemaLess) = if (colTypeList != null) {
    +          // Typed return columns.
    +          (createStructType(colTypeList).toAttributes, false)
    +        } else if (identifierSeq != null) {
    +          // Untyped return columns.
    +          val attrs = visitIdentifierSeq(identifierSeq).map { name =>
    +            AttributeReference(name, StringType, nullable = true)()
    +          }
    +          (attrs, false)
    +        } else {
    +          (Seq(AttributeReference("key", StringType)(),
    +            AttributeReference("value", StringType)()), true)
    +        }
    +
    +        // Create the transform.
    +        ScriptTransformation(
    +          expressions,
    +          string(script),
    +          attributes,
    +          withFilter,
    +          withScriptIOSchema(inRowFormat, recordWriter, outRowFormat, recordReader, schemaLess))
    +
    +      case SqlBaseParser.SELECT =>
    +        // Regular select
    +
    +        // Add lateral views.
    +        val withLateralView = ctx.lateralView.asScala.foldLeft(relation)(withGenerate)
    +
    +        // Add where.
    +        val withFilter = withLateralView.optionalMap(where)(filter)
    +
    +        // Add aggregation or a project.
    +        val namedExpressions = expressions.map {
    +          case e: NamedExpression => e
    +          case e: Expression => UnresolvedAlias(e)
    +        }
    +        val withProject = if (aggregation != null) {
    +          withAggregation(aggregation, namedExpressions, withFilter)
    +        } else if (namedExpressions.nonEmpty) {
    +          Project(namedExpressions, withFilter)
    +        } else {
    +          withFilter
    +        }
    +
    +        // Having
    +        val withHaving = withProject.optional(having) {
    +          // Note that we added a cast to boolean. If the expression itself is already boolean,
    +          // the optimizer will get rid of the unnecessary cast.
    +          Filter(Cast(expression(having), BooleanType), withProject)
    +        }
    +
    +        // Distinct
    +        val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) {
    +          Distinct(withHaving)
    +        } else {
    +          withHaving
    +        }
    +
    +        // Window
    +        withDistinct.optionalMap(windows)(withWindows)
    +    }
    +  }
    +
    +  /**
    +   * Create a (Hive based) [[ScriptInputOutputSchema]].
    +   */
    +  protected def withScriptIOSchema(
    +      inRowFormat: RowFormatContext,
    +      recordWriter: Token,
    +      outRowFormat: RowFormatContext,
    +      recordReader: Token,
    +      schemaLess: Boolean): ScriptInputOutputSchema = null
    +
    +  /**
    +   * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
    +   * separated) relations here, these get converted into a single plan by condition-less inner join.
    +   */
    +  override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
    +    val from = ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None))
    +    ctx.lateralView.asScala.foldLeft(from)(withGenerate)
    +  }
    +
    +  /**
    +   * Connect two queries by a Set operator.
    +   *
    +   * Supported Set operators are:
    +   * - UNION [DISTINCT]
    +   * - UNION ALL
    +   * - EXCEPT [DISTINCT]
    +   * - INTERSECT [DISTINCT]
    +   */
    +  override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
    +    val left = plan(ctx.left)
    +    val right = plan(ctx.right)
    +    val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.UNION if all =>
    +        Union(left, right)
    +      case SqlBaseParser.UNION =>
    +        Distinct(Union(left, right))
    +      case SqlBaseParser.INTERSECT if all =>
    +        throw new ParseException("INTERSECT ALL is not supported.", ctx)
    +      case SqlBaseParser.INTERSECT =>
    +        Intersect(left, right)
    +      case SqlBaseParser.EXCEPT if all =>
    +        throw new ParseException("EXCEPT ALL is not supported.", ctx)
    +      case SqlBaseParser.EXCEPT =>
    +        Except(left, right)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[WithWindowDefinition]] operator to a logical plan.
    +   */
    +  private def withWindows(
    +      ctx: WindowsContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Collect all window specifications defined in the WINDOW clause.
    +    val baseWindowMap = ctx.namedWindow.asScala.map {
    +      wCtx =>
    +        (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
    +    }.toMap
    +
    +    // Handle cases like
    +    // window w1 as (partition by p_mfgr order by p_name
    +    //               range between 2 preceding and 2 following),
    +    //        w2 as w1
    +    val windowMapView = baseWindowMap.mapValues {
    +      case WindowSpecReference(name) =>
    +        baseWindowMap.get(name) match {
    +          case Some(spec: WindowSpecDefinition) =>
    +            spec
    +          case Some(ref) =>
    +            throw new ParseException(s"Window reference '$name' is not a window specification", ctx)
    +          case None =>
    +            throw new ParseException(s"Cannot resolve window reference '$name'", ctx)
    +        }
    +      case spec: WindowSpecDefinition => spec
    +    }
    +
    +    // Note that mapValues creates a view instead of materialized map. We force materialization by
    +    // mapping over identity.
    +    WithWindowDefinition(windowMapView.map(identity), query)
    +  }
    +
    +  /**
    +   * Add an [[Aggregate]] to a logical plan.
    +   */
    +  private def withAggregation(
    +      ctx: AggregationContext,
    +      selectExpressions: Seq[NamedExpression],
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    val groupByExpressions = expressionList(groupingExpressions)
    +
    +    if (GROUPING != null) {
    +      // GROUP BY .... GROUPING SETS (...)
    +      val expressionMap = groupByExpressions.zipWithIndex.toMap
    +      val numExpressions = expressionMap.size
    +      val mask = (1 << numExpressions) - 1
    +      val masks = ctx.groupingSet.asScala.map {
    +        _.expression.asScala.foldLeft(mask) {
    +          case (bitmap, eCtx) =>
    +            // Find the index of the expression.
    +            val e = typedVisit[Expression](eCtx)
    +            val index = expressionMap.find(_._1.semanticEquals(e)).map(_._2).getOrElse(
    +              throw new ParseException(
    +                s"$e doesn't show up in the GROUP BY list", ctx))
    +            // 0 means that the column at the given index is a grouping column, 1 means it is not,
    +            // so we unset the bit in bitmap.
    +            bitmap & ~(1 << (numExpressions - 1 - index))
    +        }
    +      }
    +      GroupingSets(masks, groupByExpressions, query, selectExpressions)
    +    } else {
    +      // GROUP BY .... (WITH CUBE | WITH ROLLUP)?
    +      val mappedGroupByExpressions = if (CUBE != null) {
    +        Seq(Cube(groupByExpressions))
    +      } else if (ROLLUP != null) {
    +        Seq(Rollup(groupByExpressions))
    +      } else {
    +        groupByExpressions
    +      }
    +      Aggregate(mappedGroupByExpressions, selectExpressions, query)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[Generate]] (Lateral View) to a logical plan.
    +   */
    +  private def withGenerate(
    +      query: LogicalPlan,
    +      ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
    +    val expressions = expressionList(ctx.expression)
    +
    +    // Create the generator.
    +    val generator = ctx.qualifiedName.getText.toLowerCase match {
    +      case "explode" if expressions.size == 1 =>
    +        Explode(expressions.head)
    +      case "json_tuple" =>
    +        JsonTuple(expressions)
    +      case other =>
    +        withGenerator(other, expressions, ctx)
    +    }
    +
    +    Generate(
    +      generator,
    +      join = true,
    +      outer = ctx.OUTER != null,
    +      Some(ctx.tblName.getText.toLowerCase),
    +      ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply),
    +      query)
    +  }
    +
    +  /**
    +   * Create a [[Generator]]. Override this method in order to support custom Generators.
    +   */
    +  protected def withGenerator(
    +      name: String,
    +      expressions: Seq[Expression],
    +      ctx: LateralViewContext): Generator = {
    +    throw new ParseException(s"Generator function '$name' is not supported", ctx)
    +  }
    +
    +  /**
    +   * Create a joins between two or more logical plans.
    +   */
    +  override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) {
    +    /** Build a join between two plans. */
    +    def join(ctx: JoinRelationContext, left: LogicalPlan, right: LogicalPlan): Join = {
    +      val baseJoinType = ctx.joinType match {
    +        case null => Inner
    +        case jt if jt.FULL != null => FullOuter
    +        case jt if jt.SEMI != null => LeftSemi
    +        case jt if jt.LEFT != null => LeftOuter
    +        case jt if jt.RIGHT != null => RightOuter
    +        case _ => Inner
    +      }
    +
    +      // Resolve the join type and join condition
    +      val (joinType, condition) = Option(ctx.joinCriteria) match {
    +        case Some(c) if c.USING != null =>
    +          val columns = c.identifier.asScala.map { column =>
    +            UnresolvedAttribute.quoted(column.getText)
    +          }
    +          (UsingJoin(baseJoinType, columns), None)
    +        case Some(c) if c.booleanExpression != null =>
    +          (baseJoinType, Option(expression(c.booleanExpression)))
    +        case None if ctx.NATURAL != null =>
    +          (NaturalJoin(baseJoinType), None)
    +        case None =>
    +          (baseJoinType, None)
    +      }
    +      Join(left, right, joinType, condition)
    +    }
    +
    +    // Handle all consecutive join clauses. ANTLR produces a right nested tree in which the the
    +    // first join clause is at the top. However fields of previously referenced tables can be used
    +    // in following join clauses. The tree needs to be reversed in order to make this work.
    +    var result = plan(ctx.left)
    +    var current = ctx
    +    while (current != null) {
    +      current.right match {
    +        case right: JoinRelationContext =>
    +          result = join(current, result, plan(right.left))
    +          current = right
    +        case right =>
    +          result = join(current, result, plan(right))
    +          current = null
    +      }
    +    }
    +    result
    +  }
    +
    +  /**
    +   * Add a [[Sample]] to a logical plan.
    +   *
    +   * This currently supports the following sampling methods:
    +   * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
    +   * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
    +   * are defined as a number between 0 and 100.
    +   * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
    +   */
    +  private def withSample(ctx: SampleContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Create a sampled plan if we need one.
    +    def sample(fraction: Double): Sample = {
    +      // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
    +      // function takes X PERCENT as the input and the range of X is [0, 100], we need to
    +      // adjust the fraction.
    +      val eps = RandomSampler.roundingEpsilon
    +      assert(fraction >= 0.0 - eps && fraction <= 1.0 + eps,
    +        s"Sampling fraction ($fraction) must be on interval [0, 1]",
    +        ctx)
    +      Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)(true)
    +    }
    +
    +    ctx.sampleType.getType match {
    +      case SqlBaseParser.ROWS =>
    +        Limit(expression(ctx.expression), query)
    +
    +      case SqlBaseParser.PERCENTLIT =>
    +        val fraction = ctx.percentage.getText.toDouble
    +        sample(fraction / 100.0d)
    +
    +      case SqlBaseParser.BUCKET if ctx.ON != null =>
    +        throw new ParseException("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx)
    +
    +      case SqlBaseParser.BUCKET =>
    +        sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a sub-query.
    +   */
    +  override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryNoWith)
    +  }
    +
    +  /**
    +   * Create an un-aliased table reference. This is typically used for top-level table references,
    +   * for example:
    +   * {{{
    +   *   INSERT INTO db.tbl2
    +   *   TABLE db.tbl1
    +   * }}}
    +   */
    +  override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None)
    +  }
    +
    +  /**
    +   * Create an aliased table reference. This is typically used in FROM clauses.
    +   */
    +  override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
    +    val table = UnresolvedRelation(
    +      visitTableIdentifier(ctx.tableIdentifier),
    +      Option(ctx.identifier).map(_.getText))
    +    table.optionalMap(ctx.sample)(withSample)
    +  }
    +
    +  /**
    +   * Create an inline table (a virtual table in Hive parlance).
    +   */
    +  override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
    +    // Get the backing expressions.
    +    val expressions = ctx.expression.asScala.map { eCtx =>
    +      val e = expression(eCtx)
    +      assert(e.foldable, "All expressions in an inline table must be constants.", eCtx)
    +      e
    +    }
    +
    +    // Validate and evaluate the rows.
    +    val (structType, structConstructor) = expressions.head.dataType match {
    --- End diff --
    
    As discussed offline: The current run is to use the schema of the first row and to force other rows to comply to that schema. This is semantically very different from the same thing written as a number of UNION ALL statements, e.g.: `SELECT 1 AS id , 'a' AS name UNION ALL SELECT 2, 'b' UNION ALL SELECT 2, 'c'`
    
    Fixing this has not the highest priority. The fix should probably be using `UNION ALL`s instead of a `LocalRelation` (the Optimizer should be able to rewrite this into a LocalRelation eventually).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195806454
  
    **[Test build #53011 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53011/consoleFull)** for PR 11557 at commit [`5d4e13f`](https://github.com/apache/spark/commit/5d4e13f0a5161b6431936641649599e3f8f37134).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193426807
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/52572/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57272561
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    +    assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b)))
    +    assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b))
    +  }
    +
    +  test("transform query spec") {
    +    val p = ScriptTransformation(Seq('a, 'b), "func", Seq.empty, table("e"), null)
    +    assertEqual("select transform(a, b) using 'func' from e where f < 10",
    +      p.copy(child = p.child.where('f < 10)))
    +    assertEqual("map(a, b) using 'func' as c, d from e",
    +      p.copy(output = Seq('c.string, 'd.string)))
    +    assertEqual("reduce(a, b) using 'func' as (c: int, d decimal(10, 0)) from e",
    +      p.copy(output = Seq('c.int, 'd.decimal(10, 0))))
    +  }
    +
    +  test("multi select query") {
    +    assertEqual(
    +      "from a select * select * where s < 10",
    +      table("a").select(star()).unionAll(table("a").where('s < 10).select(star())))
    +    intercept(
    +      "from a select * select * from x where a.s < 10",
    +      "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements")
    +    assertEqual(
    +      "from a insert into tbl1 select * insert into tbl2 select * where s < 10",
    +      table("a").select(star()).insertInto("tbl1").unionAll(
    +        table("a").where('s < 10).select(star()).insertInto("tbl2")))
    +  }
    +
    +  test("query organization") {
    +    // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows
    +    val baseSql = "select * from t"
    +    val basePlan = table("t").select(star())
    +
    +    val ws = Map("w1" -> WindowSpecDefinition(Seq.empty, Seq.empty, UnspecifiedFrame))
    +    val limitWindowClauses = Seq(
    +      ("", (p: LogicalPlan) => p),
    +      (" limit 10", (p: LogicalPlan) => p.limit(10)),
    +      (" window w1 as ()", (p: LogicalPlan) => WithWindowDefinition(ws, p)),
    +      (" window w1 as () limit 10", (p: LogicalPlan) => WithWindowDefinition(ws, p).limit(10))
    +    )
    +
    +    val orderSortDistrClusterClauses = Seq(
    +      ("", basePlan),
    +      (" order by a, b desc", basePlan.orderBy('a.asc, 'b.desc)),
    +      (" sort by a, b desc", basePlan.sortBy('a.asc, 'b.desc)),
    +      (" distribute by a, b", basePlan.distribute('a, 'b)),
    +      (" distribute by a sort by b", basePlan.distribute('a).sortBy('b.asc)),
    +      (" cluster by a, b", basePlan.distribute('a, 'b).sortBy('a.asc, 'b.asc))
    +    )
    +
    +    orderSortDistrClusterClauses.foreach {
    +      case (s1, p1) =>
    +        limitWindowClauses.foreach {
    +          case (s2, pf2) =>
    +            assertEqual(baseSql + s1 + s2, pf2(p1))
    +        }
    +    }
    +
    +    val msg = "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported"
    +    intercept(s"$baseSql order by a sort by a", msg)
    +    intercept(s"$baseSql cluster by a distribute by a", msg)
    +    intercept(s"$baseSql order by a cluster by a", msg)
    +    intercept(s"$baseSql order by a distribute by a", msg)
    +  }
    +
    +  test("insert into") {
    +    val sql = "select * from t"
    +    val plan = table("t").select(star())
    +    def insert(
    +        partition: Map[String, Option[String]],
    +        overwrite: Boolean = false,
    +        ifNotExists: Boolean = false): LogicalPlan =
    +      InsertIntoTable(table("s"), partition, plan, overwrite, ifNotExists)
    +
    +    // Single inserts
    +    assertEqual(s"insert overwrite table s $sql",
    +      insert(Map.empty, overwrite = true))
    +    assertEqual(s"insert overwrite table s if not exists $sql",
    +      insert(Map.empty, overwrite = true, ifNotExists = true))
    +    assertEqual(s"insert into s $sql",
    +      insert(Map.empty))
    +    assertEqual(s"insert into table s partition (c = 'd', e = 1) $sql",
    +      insert(Map("c" -> Option("d"), "e" -> Option("1"))))
    +    assertEqual(s"insert overwrite table s partition (c = 'd', x) if not exists $sql",
    +      insert(Map("c" -> Option("d"), "x" -> None), overwrite = true, ifNotExists = true))
    +
    +    // Multi insert
    +    val plan2 = table("t").where('x > 5).select(star())
    +    assertEqual("from t insert into s select * limit 1 insert into u select * where x > 5",
    +      InsertIntoTable(
    +        table("s"), Map.empty, plan.limit(1), overwrite = false, ifNotExists = false).unionAll(
    +        InsertIntoTable(
    +          table("u"), Map.empty, plan2, overwrite = false, ifNotExists = false)))
    +  }
    +
    +  test("aggregation") {
    +    val sql = "select a, b, sum(c) as c from d group by a, b"
    +
    +    // Normal
    +    assertEqual(sql, table("d").groupBy('a, 'b)('a, 'b, 'sum.function('c).as("c")))
    +
    +    // Cube
    +    assertEqual(s"$sql with cube",
    +      table("d").groupBy(Cube(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
    +
    +    // Rollup
    +    assertEqual(s"$sql with rollup",
    +      table("d").groupBy(Rollup(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
    +
    +    // Grouping Sets
    +    assertEqual(s"$sql grouping sets((a, b), (a), ())",
    +      GroupingSets(Seq(0, 1, 3), Seq('a, 'b), table("d"), Seq('a, 'b, 'sum.function('c).as("c"))))
    +    intercept(s"$sql grouping sets((a, b), (c), ())",
    +      "c doesn't show up in the GROUP BY list")
    +  }
    +
    +  test("limit") {
    +    val sql = "select * from t"
    +    val plan = table("t").select(star())
    +    assertEqual(s"$sql limit 10", plan.limit(10))
    +    assertEqual(s"$sql limit cast(9 / 4 as int)", plan.limit(Cast(Literal(9) / 4, IntegerType)))
    +  }
    +
    +  test("window spec") {
    +    // Note that WindowSpecs are testing in the ExpressionParserSuite
    +    val sql = "select * from t"
    +    val plan = table("t").select(star())
    +    val spec = WindowSpecDefinition(Seq('a, 'b), Seq('c.asc),
    +      SpecifiedWindowFrame(RowFrame, ValuePreceding(1), ValueFollowing(1)))
    +
    +    // Test window resolution.
    +    val ws1 = Map("w1" -> spec, "w2" -> spec, "w3" -> spec)
    +    assertEqual(
    +      s"""$sql
    +         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
    +         |       w2 as w1,
    +         |       w3 as w1""".stripMargin,
    +      WithWindowDefinition(ws1, plan))
    +
    +    // Fail with no reference.
    +    intercept(s"$sql window w2 as w1", "Cannot resolve window reference 'w1'")
    +
    +    // Fail when resolved reference is not a window spec.
    +    intercept(
    +      s"""$sql
    +         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
    +         |       w2 as w1,
    +         |       w3 as w2""".stripMargin,
    +      "Window reference 'w2' is not a window specification"
    +    )
    +  }
    +
    +  test("lateral view") {
    +    // Single lateral view
    +    assertEqual(
    +      "select * from t lateral view explode(x) expl as x",
    +      table("t")
    +        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
    +        .select(star()))
    +
    +    // Multiple lateral views
    +    assertEqual(
    +      """select *
    +        |from t
    +        |lateral view explode(x) expl as x
    +        |lateral view outer json_tuple(x, y) jtup q, z""".stripMargin,
    +      table("t")
    +        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
    +        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = true, Some("jtup"), Seq("q", "z"))
    +        .select(star()))
    +
    +    // Multi-Insert lateral views.
    +    val from = table("t1").generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
    +    assertEqual(
    +      """from t1
    +        |lateral view explode(x) expl as x
    +        |insert into t2
    +        |select *
    +        |lateral view json_tuple(x, y) jtup q, z
    +        |insert into t3
    +        |select *
    +        |where s < 10
    +      """.stripMargin,
    +      Union(from
    +        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = false, Some("jtup"), Seq("q", "z"))
    +        .select(star())
    +        .insertInto("t2"),
    +        from.where('s < 10).select(star()).insertInto("t3")))
    +
    +    // Unsupported generator.
    +    intercept(
    +      "select * from t lateral view posexplode(x) posexpl as x, y",
    +      "Generator function 'posexplode' is not supported")
    +  }
    +
    +  test("joins") {
    +    val testUnconditionalJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t as tt $sql u",
    +        table("t").as("tt").join(table("u"), jt, None).select(star()))
    +    }
    +    val testConditionalJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t $sql u as uu on a = b",
    +        table("t").join(table("u").as("uu"), jt, Option('a === 'b)).select(star()))
    +    }
    +    val testNaturalJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t tt natural $sql u as uu",
    +        table("t").as("tt").join(table("u").as("uu"), NaturalJoin(jt), None).select(star()))
    +    }
    +    val testUsingJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t $sql u using(a, b)",
    +        table("t").join(table("u"), UsingJoin(jt, Seq('a.attr, 'b.attr)), None).select(star()))
    +    }
    +    val testAll = Seq(testUnconditionalJoin, testConditionalJoin, testNaturalJoin, testUsingJoin)
    +
    +    def test(sql: String, jt: JoinType, tests: Seq[(String, JoinType) => Unit]): Unit = {
    +      tests.foreach(_(sql, jt))
    +    }
    +    test("cross join", Inner, Seq(testUnconditionalJoin))
    +    test(",", Inner, Seq(testUnconditionalJoin))
    +    test("join", Inner, testAll)
    +    test("inner join", Inner, testAll)
    +    test("left join", LeftOuter, testAll)
    +    test("left outer join", LeftOuter, testAll)
    +    test("right join", RightOuter, testAll)
    +    test("right outer join", RightOuter, testAll)
    +    test("full join", FullOuter, testAll)
    +    test("full outer join", FullOuter, testAll)
    +  }
    +
    +  test("sampled relations") {
    +    val sql = "select * from t"
    +    assertEqual(s"$sql tablesample(100 rows)",
    +      table("t").limit(100).select(star()))
    +    assertEqual(s"$sql as x tablesample(43 percent)",
    +      Sample(0, .43d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
    +    assertEqual(s"$sql as x tablesample(bucket 4 out of 10)",
    +      Sample(0, .4d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
    +    intercept(s"$sql as x tablesample(bucket 4 out of 10 on x)",
    +      "TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported")
    +    intercept(s"$sql as x tablesample(bucket 11 out of 10)",
    +      s"Sampling fraction (${11.0/10.0}) must be on interval [0, 1]")
    +  }
    +
    +  test("sub-query") {
    +    val plan = table("t0").select('id)
    +    assertEqual("select id from (t0)", plan)
    +    assertEqual("select id from ((((((t0))))))", plan)
    +    assertEqual(
    +      "(select * from t1) union distinct (select * from t2)",
    +      Distinct(table("t1").select(star()).unionAll(table("t2").select(star()))))
    +    assertEqual(
    +      "select * from ((select * from t1) union (select * from t2)) t",
    +      Distinct(
    +        table("t1").select(star()).unionAll(table("t2").select(star()))).as("t").select(star()))
    +    assertEqual(
    +      """select  id
    +        |from (((select id from t0)
    +        |       union all
    +        |       (select  id from t0))
    +        |      union all
    +        |      (select id from t0)) as u_1
    +      """.stripMargin,
    +      plan.unionAll(plan).unionAll(plan).as("u_1").select('id))
    +  }
    +
    +  test("scalar sub-query") {
    +    assertEqual(
    +      "select (select max(b) from s) ss from t",
    +      table("t").select(ScalarSubquery(table("s").select('max.function('b))).as("ss")))
    +    assertEqual(
    +      "select * from t where a = (select b from s)",
    +      table("t").where('a === ScalarSubquery(table("s").select('b))).select(star()))
    +    assertEqual(
    +      "select g from t group by g having a > (select b from s)",
    +      table("t").groupBy('g)('g).where('a > ScalarSubquery(table("s").select('b))))
    +  }
    +
    +  test("table reference") {
    +    assertEqual("table t", table("t"))
    +    assertEqual("table d.t", table("d", "t"))
    +  }
    +
    +  test("inline table") {
    +    assertEqual("values 1, 2, 3, 4", LocalRelation.fromExternalRows(
    --- End diff --
    
    This is interesting, do you have a complete example for this syntax?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57269652
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -0,0 +1,250 @@
    +/*
    + * 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.execution
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.misc.Interval
    +
    +import org.apache.spark.sql.SaveMode
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.parser.ng.{AbstractSqlParser, AstBuilder}
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.execution.command._
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * Concrete parser for Spark SQL statements.
    + */
    +object SparkSqlParser extends AbstractSqlParser{
    +  val astBuilder = new SparkSqlAstBuilder
    +}
    +
    +/**
    + * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier.
    + */
    +class SparkSqlAstBuilder extends AstBuilder {
    +  import AstBuilder._
    +  import org.apache.spark.sql.catalyst.parser.ParseUtils._
    +
    +  /**
    +   * Create a [[SetCommand]] logical plan.
    +   *
    +   * Note that we assume that everything after the SET keyword is assumed to be a part of the
    +   * key-value pair. The split between key and value is made by searching for the first `=`
    +   * character in the raw string.
    +   */
    +  override def visitSetConfiguration(ctx: SetConfigurationContext): LogicalPlan = withOrigin(ctx) {
    --- End diff --
    
    So we have some grammars that are only allowed in sql module, not catalyst module. What if we use these grammars at catalyst module? Will we throw parser exception?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198167945
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202481169
  
    **[Test build #54318 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54318/consoleFull)** for PR 11557 at commit [`4200ad6`](https://github.com/apache/spark/commit/4200ad670d6c6d137f2f43470d744b1d9087cbed).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  implicit class EnhancedLogicalPlan(val plan: LogicalPlan) extends AnyVal `


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198889006
  
    **[Test build #53631 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53631/consoleFull)** for PR 11557 at commit [`4f1da75`](https://github.com/apache/spark/commit/4f1da75388424c6d4169af900655c769524bee02).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195881884
  
    **[Test build #53020 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53020/consoleFull)** for PR 11557 at commit [`303394f`](https://github.com/apache/spark/commit/303394f6c114eda0c085c871dd34b7ae70e9c81f).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193554969
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/52613/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-199785654
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202481762
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193704464
  
    **[Test build #52646 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52646/consoleFull)** for PR 11557 at commit [`2c9a092`](https://github.com/apache/spark/commit/2c9a092e2c3c944ecfeaa9dbdd55fd2665fb3d23).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class ParseException(CapturedException):`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202558809
  
    looks like we are missing antlr4 maven plugin.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202420516
  
    **[Test build #54318 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54318/consoleFull)** for PR 11557 at commit [`4200ad6`](https://github.com/apache/spark/commit/4200ad670d6c6d137f2f43470d744b1d9087cbed).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193368150
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/52567/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57168260
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -80,7 +80,7 @@ private[sql] class SessionState(ctx: SQLContext) {
       /**
        * Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
        */
    -  lazy val sqlParser: ParserInterface = new SparkQl(conf)
    +  lazy val sqlParser: ParserInterface = SparkSqlParser
    --- End diff --
    
    It is still in use in Hive. I'll open up a follow-up PR in the following week or so which will also parse the Hive commands.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-199734370
  
    **[Test build #53764 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53764/consoleFull)** for PR 11557 at commit [`a5d12ba`](https://github.com/apache/spark/commit/a5d12ba93a3c29194800d5e7de11704a96741833).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57149320
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/ng/SqlBase.g4 ---
    @@ -0,0 +1,742 @@
    +/*
    + * Licensed 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.
    + *
    + * This file is an adaptation of Presto's presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 grammar.
    + */
    +
    +grammar SqlBase;
    +
    +tokens {
    +    DELIMITER
    +}
    +
    +singleStatement
    +    : statement EOF
    +    ;
    +
    +singleExpression
    +    : namedExpression EOF
    +    ;
    +
    +singleTableIdentifier
    +    : tableIdentifier EOF
    +    ;
    +
    +singleDataType
    +    : dataType EOF
    +    ;
    +
    +statement
    +    : query                                                            #statementDefault
    +    | USE db=identifier                                                #use
    +    | createTable ('(' colTypeList ')')? tableProvider tableProperties #createTableUsing
    +    | createTable tableProvider tableProperties? AS? query             #createTableUsingAsSelect
    +    | DROP TABLE (IF EXISTS)? qualifiedName                            #dropTable
    +    | DELETE FROM qualifiedName (WHERE booleanExpression)?             #delete
    +    | ALTER TABLE from=qualifiedName RENAME TO to=qualifiedName        #renameTable
    +    | ALTER TABLE tableName=qualifiedName
    +        RENAME COLUMN from=identifier TO to=identifier                 #renameColumn
    +    | ALTER TABLE tableName=qualifiedName
    +        ADD COLUMN column=colType                                      #addColumn
    +    | CREATE (OR REPLACE)? VIEW qualifiedName AS query                 #createView
    +    | DROP VIEW (IF EXISTS)? qualifiedName                             #dropView
    +    | CALL qualifiedName '(' (callArgument (',' callArgument)*)? ')'   #call
    +    | EXPLAIN explainOption* statement                                 #explain
    +    | SHOW TABLES ((FROM | IN) db=identifier)?
    +        (LIKE (qualifiedName | pattern=STRING))?                       #showTables
    +    | SHOW SCHEMAS ((FROM | IN) identifier)?                           #showSchemas
    +    | SHOW CATALOGS                                                    #showCatalogs
    +    | SHOW COLUMNS (FROM | IN) qualifiedName                           #showColumns
    +    | SHOW FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))?         #showFunctions
    +    | (DESC | DESCRIBE) FUNCTION EXTENDED? qualifiedName               #describeFunction
    +    | (DESC | DESCRIBE) option=(EXTENDED | FORMATTED)?
    +        tableIdentifier partitionSpec? describeColName?                #describeTable
    +    | SHOW SESSION                                                     #showSession
    +    | SET SESSION qualifiedName EQ expression                          #setSession
    +    | RESET SESSION qualifiedName                                      #resetSession
    +    | START TRANSACTION (transactionMode (',' transactionMode)*)?      #startTransaction
    +    | COMMIT WORK?                                                     #commit
    +    | ROLLBACK WORK?                                                   #rollback
    +    | SHOW PARTITIONS (FROM | IN) qualifiedName
    +        (WHERE booleanExpression)?
    +        (ORDER BY sortItem (',' sortItem)*)?
    +        (LIMIT limit=(INTEGER_VALUE | ALL))?                           #showPartitions
    +    | REFRESH TABLE tableIdentifier                                    #refreshTable
    +    | CACHE LAZY? TABLE identifier (AS? query)?                        #cacheTable
    +    | UNCACHE TABLE identifier                                         #uncacheTable
    +    | CLEAR CACHE                                                      #clearCache
    +    | SET .*?                                                          #setConfiguration
    +    ;
    +
    +createTable
    +    : CREATE TEMPORARY? TABLE (IF NOT EXISTS)? tableIdentifier
    +    ;
    +
    +query
    +    : ctes? queryNoWith
    +    ;
    +
    +insertInto
    +    : INSERT OVERWRITE TABLE tableIdentifier partitionSpec? (IF NOT EXISTS)?
    +    | INSERT INTO TABLE? tableIdentifier partitionSpec?
    +    ;
    +
    +partitionSpec
    +    : PARTITION '(' partitionVal (',' partitionVal)* ')'
    +    ;
    +
    +partitionVal
    +    : identifier (EQ constant)?
    +    ;
    +
    +describeColName
    +    : identifier ('.' (identifier | STRING))*
    +    ;
    +
    +ctes
    +    : WITH namedQuery (',' namedQuery)*
    +    ;
    +
    +namedQuery
    +    : name=identifier AS? '(' queryNoWith ')'
    +    ;
    +
    +tableProvider
    +    : USING qualifiedName
    +    ;
    +
    +tableProperties
    +    :(OPTIONS | WITH) '(' tableProperty (',' tableProperty)* ')'
    +    ;
    +
    +tableProperty
    +    : key=tablePropertyKey (EQ? value=STRING)?
    +    ;
    +
    +tablePropertyKey
    +    : looseIdentifier ('.' looseIdentifier)*
    +    | STRING
    +    ;
    +
    +queryNoWith
    +    : insertInto? queryTerm queryOrganization                                              #singleInsertQuery
    +    | fromClause multiInsertQueryBody+                                                     #multiInsertQuery
    +    ;
    +
    +queryOrganization
    +    : (ORDER BY order+=sortItem (',' order+=sortItem)*)?
    +      (CLUSTER BY clusterBy+=expression (',' clusterBy+=expression)*)?
    +      (DISTRIBUTE BY distributeBy+=expression (',' distributeBy+=expression)*)?
    +      (SORT BY sort+=sortItem (',' sort+=sortItem)*)?
    +      windows?
    +      (LIMIT limit=expression)?
    +    ;
    +
    +multiInsertQueryBody
    +    : insertInto?
    +      querySpecification
    +      queryOrganization
    +    ;
    +
    +queryTerm
    +    : queryPrimary                                                                         #queryTermDefault
    +    | left=queryTerm operator=(INTERSECT | UNION | EXCEPT) setQuantifier? right=queryTerm  #setOperation
    +    ;
    +
    +queryPrimary
    +    : querySpecification                                                    #queryPrimaryDefault
    +    | TABLE tableIdentifier                                                 #table
    +    | inlineTable                                                           #inlineTableDefault1
    +    | '(' queryNoWith  ')'                                                  #subquery
    +    ;
    +
    +sortItem
    +    : expression ordering=(ASC | DESC)?
    +    ;
    +
    +querySpecification
    +    : (((SELECT kind=TRANSFORM | kind=MAP | kind=REDUCE)) '(' namedExpression (',' namedExpression)* ')'
    +       inRowFormat=rowFormat?
    +       USING script=STRING
    +       (AS (columnAliasList | colTypeList | ('(' (columnAliasList | colTypeList) ')')))?
    +       outRowFormat=rowFormat?
    +       (RECORDREADER outRecordReader=STRING)?
    +       fromClause?
    +       (WHERE where=booleanExpression)?)
    +    | (kind=SELECT setQuantifier? namedExpression (',' namedExpression)*
    +       fromClause?
    +       lateralView*
    +       (WHERE where=booleanExpression)?
    +       aggregation?
    +       (HAVING having=booleanExpression)?
    +       windows?)
    +    ;
    +
    +fromClause
    +    : FROM relation (',' relation)* lateralView*
    +    ;
    +
    +aggregation
    +    : GROUP BY groupingExpressions+=expression (',' groupingExpressions+=expression)* (
    +      WITH kind=ROLLUP
    +    | WITH kind=CUBE
    +    | kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')')?
    +    ;
    +
    +groupingSet
    +    : '(' (expression (',' expression)*)? ')'
    +    | expression
    +    ;
    +
    +lateralView
    +    : LATERAL VIEW (OUTER)? qualifiedName '(' (expression (',' expression)*)? ')' tblName=identifier (AS? colName+=identifier (',' colName+=identifier)*)
    +    ;
    +
    +setQuantifier
    +    : DISTINCT
    +    | ALL
    +    ;
    +
    +relation
    +    : left=relation
    +      ( CROSS JOIN right=sampledRelation
    +      | joinType JOIN rightRelation=relation joinCriteria?
    +      | NATURAL joinType JOIN right=sampledRelation
    +      )                                           #joinRelation
    +    | sampledRelation                             #relationDefault
    +    ;
    +
    +joinType
    +    : INNER?
    +    | LEFT OUTER?
    +    | LEFT SEMI
    +    | RIGHT OUTER?
    +    | FULL OUTER?
    +    ;
    +
    +joinCriteria
    +    : ON booleanExpression
    +    | USING '(' identifier (',' identifier)* ')'
    +    ;
    +
    +sampledRelation
    +    : relationPrimary (
    +        TABLESAMPLE '('
    +         ( (percentage=(INTEGER_VALUE | DECIMAL_VALUE) sampleType=PERCENTLIT)
    +         | (expression sampleType=ROWS)
    +         | (sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE (ON identifier)?))
    +         ')'
    +      )?
    +    ;
    +
    +columnAliases
    +    : '(' columnAliasList ')'
    +    ;
    +
    +columnAliasList
    +    : identifier (',' identifier)*
    +    ;
    +
    +relationPrimary
    +    : tableIdentifier (AS? identifier)?                             #tableName
    +    | '(' queryNoWith ')' (AS? identifier)?                         #aliasedQuery
    +    | '(' relation ')'  (AS? identifier)?                           #aliasedRelation
    +    | inlineTable                                                   #inlineTableDefault2
    +    ;
    +
    +inlineTable
    +    : VALUES expression (',' expression)*  (AS? identifier columnAliases?)?
    +    ;
    +
    +rowFormat
    +    : rowFormatSerde
    +    | rowFormatDelimited
    +    ;
    +
    +rowFormatSerde
    +    : ROW FORMAT SERDE name=STRING (WITH SERDEPROPERTIES props=tableProperties)?
    +    ;
    +
    +rowFormatDelimited
    +    : ROW FORMAT DELIMITED
    +      (FIELDS TERMINATED BY fieldsTerminatedBy=STRING)?
    +      (COLLECTION ITEMS TERMINATED BY collectionItemsTerminatedBy=STRING)?
    +      (MAP KEYS TERMINATED BY keysTerminatedBy=STRING)?
    +      (ESCAPED BY escapedBy=STRING)?
    +      (LINES SEPARATED BY linesSeparatedBy=STRING)?
    +    ;
    +
    +tableIdentifier
    +    : (db=identifier '.')? table=identifier
    +    ;
    +
    +namedExpression
    +    : expression (AS? (identifier | columnAliases))?
    +    ;
    +
    +expression
    +    : booleanExpression
    +    ;
    +
    +booleanExpression
    +    : predicated                                                   #booleanDefault
    +    | NOT booleanExpression                                        #logicalNot
    +    | left=booleanExpression operator=AND right=booleanExpression  #logicalBinary
    +    | left=booleanExpression operator=OR right=booleanExpression   #logicalBinary
    +    | EXISTS '(' query ')'                                         #exists
    +    ;
    +
    +// workaround for:
    +//  https://github.com/antlr/antlr4/issues/780
    +//  https://github.com/antlr/antlr4/issues/781
    +predicated
    +    : valueExpression predicate[$valueExpression.ctx]?
    +    ;
    +
    +predicate[ParserRuleContext value]
    +    : comparisonOperator right=valueExpression                            #comparison
    +    | NOT? BETWEEN lower=valueExpression AND upper=valueExpression        #between
    +    | NOT? IN '(' expression (',' expression)* ')'                        #inList
    +    | NOT? IN '(' query ')'                                               #inSubquery
    +    | NOT? like=(RLIKE | LIKE) pattern=valueExpression                    #like
    +    | IS NOT? NULL                                                        #nullPredicate
    +    ;
    +
    +valueExpression
    +    : primaryExpression                                                                      #valueExpressionDefault
    +    | operator=(MINUS | PLUS | TILDE) valueExpression                                        #arithmeticUnary
    +    | left=valueExpression operator=(ASTERISK | SLASH | PERCENT | DIV) right=valueExpression #arithmeticBinary
    +    | left=valueExpression operator=(PLUS | MINUS) right=valueExpression                     #arithmeticBinary
    +    | left=valueExpression operator=AMPERSAND right=valueExpression                          #arithmeticBinary
    +    | left=valueExpression operator=HAT right=valueExpression                                #arithmeticBinary
    +    | left=valueExpression operator=PIPE right=valueExpression                               #arithmeticBinary
    +    ;
    +
    +primaryExpression
    +    : constant                                                                                 #constantDefault
    +    | ASTERISK                                                                                 #star
    +    | qualifiedName '.' ASTERISK                                                               #star
    +    | '(' expression (',' expression)+ ')'                                                     #rowConstructor
    +    | qualifiedName '(' (ASTERISK) ')' (OVER windowSpec)?                                      #functionCall
    +    | qualifiedName '(' (setQuantifier? expression (',' expression)*)? ')' (OVER windowSpec)?  #functionCall
    +    | '(' query ')'                                                                            #subqueryExpression
    +    | CASE valueExpression whenClause+ (ELSE elseExpression=expression)? END                   #simpleCase
    +    | CASE whenClause+ (ELSE elseExpression=expression)? END                                   #searchedCase
    +    | CAST '(' expression AS dataType ')'                                                      #cast
    +    | value=primaryExpression '[' index=valueExpression ']'                                    #subscript
    +    | identifier                                                                               #columnReference
    +    | base=primaryExpression '.' fieldName=identifier                                          #dereference
    +    | '(' expression ')'                                                                       #parenthesizedExpression
    +    ;
    +
    +constant
    +    : NULL                                                                                     #nullLiteral
    +    | interval                                                                                 #intervalLiteral
    +    | identifier STRING                                                                        #typeConstructor
    +    | number                                                                                   #numericLiteral
    +    | booleanValue                                                                             #booleanLiteral
    +    | STRING+                                                                                  #stringLiteral
    +    ;
    +
    +comparisonOperator
    +    : EQ | NEQ | NEQJ | LT | LTE | GT | GTE | NSEQ
    +    ;
    +
    +booleanValue
    +    : TRUE | FALSE
    +    ;
    +
    +interval
    +    : INTERVAL intervalField*
    +    ;
    +
    +intervalField
    +    : value=intervalValue unit=identifier (TO to=identifier)?
    +    ;
    +
    +intervalValue
    +    : (PLUS | MINUS)? (INTEGER_VALUE | DECIMAL_VALUE)
    +    | STRING
    +    ;
    +
    +dataType
    +    : complex=ARRAY '<' dataType '>'                            #complexDataType
    +    | complex=MAP '<' dataType ',' dataType '>'                 #complexDataType
    +    | complex=STRUCT ('<' colTypeList? '>' | NEQ)              #complexDataType
    +    | identifier ('(' INTEGER_VALUE (',' INTEGER_VALUE)* ')')?  #primitiveDataType
    +    ;
    +
    +colTypeList
    +    : colType (',' colType)*
    +    ;
    +
    +colType
    +    : identifier ':'? dataType (COMMENT STRING)?
    +    ;
    +
    +whenClause
    +    : WHEN condition=expression THEN result=expression
    +    ;
    +
    +windows
    +    : WINDOW namedWindow (',' namedWindow)*
    +    ;
    +
    +namedWindow
    +    : identifier AS windowSpec
    +    ;
    +
    +windowSpec
    +    : name=identifier  #windowRef
    +    | '('
    +      (PARTITION BY partition+=expression (',' partition+=expression)*)?
    +      (ORDER BY sortItem (',' sortItem)* windowFrame?)?
    +      ')'              #windowDef
    +    ;
    +
    +windowFrame
    +    : frameType=RANGE start=frameBound
    +    | frameType=ROWS start=frameBound
    +    | frameType=RANGE BETWEEN start=frameBound AND end=frameBound
    +    | frameType=ROWS BETWEEN start=frameBound AND end=frameBound
    +    ;
    +
    +frameBound
    +    : UNBOUNDED boundType=(PRECEDING | FOLLOWING)
    +    | boundType=CURRENT ROW
    +    | expression boundType=(PRECEDING | FOLLOWING)
    +    ;
    +
    +
    +explainOption
    +    : LOGICAL | FORMATTED | EXTENDED
    +    ;
    +
    +transactionMode
    +    : ISOLATION LEVEL levelOfIsolation    #isolationLevel
    +    | READ accessMode=(ONLY | WRITE)      #transactionAccessMode
    +    ;
    +
    +levelOfIsolation
    +    : READ UNCOMMITTED                    #readUncommitted
    +    | READ COMMITTED                      #readCommitted
    +    | REPEATABLE READ                     #repeatableRead
    +    | SERIALIZABLE                        #serializable
    +    ;
    +
    +callArgument
    +    : expression                    #positionalArgument
    +    | identifier '=>' expression    #namedArgument
    +    ;
    +
    +qualifiedName
    +    : identifier ('.' identifier)*
    +    ;
    +
    +// Identifier that also allows the use of a number of SQL keywords (mainly for backwards compatibility).
    +looseIdentifier
    +    : identifier
    +    | FROM
    +    | TO
    +    | TABLE
    +    | WITH
    +    ;
    +
    +identifier
    +    : IDENTIFIER             #unquotedIdentifier
    +    | quotedIdentifier       #quotedIdentifierAlternative
    +    | nonReserved            #unquotedIdentifier
    +    ;
    +
    +quotedIdentifier
    +    : BACKQUOTED_IDENTIFIER
    +    ;
    +
    +number
    +    : DECIMAL_VALUE            #decimalLiteral
    +    | SCIENTIFIC_DECIMAL_VALUE #scientificDecimalLiteral
    +    | INTEGER_VALUE            #integerLiteral
    +    | BIGINT_LITERAL           #bigIntLiteral
    +    | SMALLINT_LITERAL         #smallIntLiteral
    +    | TINYINT_LITERAL          #tinyIntLiteral
    +    | DOUBLE_LITERAL           #doubleLiteral
    +    ;
    +
    +nonReserved
    +    : SHOW | TABLES | COLUMNS | COLUMN | PARTITIONS | FUNCTIONS | SCHEMAS | CATALOGS | SESSION
    +    | ADD
    +    | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | MAP | ARRAY | STRUCT
    +    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
    +    | DELIMITED | FIELDS | TERMINATED | COLLECTION | ITEMS | KEYS | ESCAPED | LINES | SEPARATED
    +    | EXTENDED | REFRESH | CLEAR | CACHE | UNCACHE | LAZY | TEMPORARY | OPTIONS
    +    | GROUPING | CUBE | ROLLUP
    +    | EXPLAIN | FORMAT | LOGICAL | FORMATTED
    +    | TABLESAMPLE | USE | TO | BUCKET | PERCENTLIT | OUT | OF
    +    | SET | RESET
    +    | VIEW | REPLACE
    +    | IF
    +    | NO | DATA
    +    | START | TRANSACTION | COMMIT | ROLLBACK | WORK | ISOLATION | LEVEL
    +    | SERIALIZABLE | REPEATABLE | COMMITTED | UNCOMMITTED | READ | WRITE | ONLY
    +    | CALL
    +    | SORT | CLUSTER | DISTRIBUTE
    +    ;
    +
    +SELECT: 'SELECT';
    --- End diff --
    
    We could. However it currently works better if they are in the same file when we are making modifications (which we currently are).
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279420
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    +    assertEqual("foo(*)", 'foo.function(1))
    +    assertEqual("foo(a, b)", 'foo.function('a, 'b))
    +    assertEqual("foo(all a, b)", 'foo.function('a, 'b))
    +    assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b))
    +    assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b))
    +    assertEqual("`select`(all a, b)", 'select.function('a, 'b))
    +  }
    +
    +  test("window function expressions") {
    +    val func = 'foo.function(1)
    +    def windowed(
    +        partitioning: Seq[Expression] = Seq.empty,
    +        ordering: Seq[SortOrder] = Seq.empty,
    +        frame: WindowFrame = UnspecifiedFrame): Expression = {
    +      WindowExpression(func, WindowSpecDefinition(partitioning, ordering, frame))
    +    }
    +
    +    // Basic window testing.
    +    assertEqual("foo(*) over w1", UnresolvedWindowExpression(func, WindowSpecReference("w1")))
    +    assertEqual("foo(*) over ()", windowed())
    +    assertEqual("foo(*) over (partition by a, b)", windowed(Seq('a, 'b)))
    +    assertEqual("foo(*) over (order by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc )))
    +    assertEqual("foo(*) over (partition by a, b order by c)", windowed(Seq('a, 'b), Seq('c.asc)))
    +
    +    // Test use of expressions in window functions.
    +    assertEqual(
    +      "sum(product + 1) over (partition by ((product) + (1)) order by 2)",
    +      WindowExpression('sum.function('product + 1),
    +        WindowSpecDefinition(Seq('product + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
    +    assertEqual(
    +      "sum(product + 1) over (partition by ((product / 2) + 1) order by 2)",
    +      WindowExpression('sum.function('product + 1),
    +        WindowSpecDefinition(Seq('product / 2 + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
    +
    +    // Range/Row
    +    val frameTypes = Seq(("rows", RowFrame), ("range", RangeFrame))
    +    val boundaries = Seq(
    +      ("10 preceding", ValuePreceding(10), CurrentRow),
    +      ("3 + 1 following", ValueFollowing(4), CurrentRow), // Will fail during analysis
    +      ("unbounded preceding", UnboundedPreceding, CurrentRow),
    +      ("unbounded following", UnboundedFollowing, CurrentRow), // Will fail during analysis
    +      ("between unbounded preceding and current row", UnboundedPreceding, CurrentRow),
    +      ("between unbounded preceding and unbounded following",
    +        UnboundedPreceding, UnboundedFollowing),
    +      ("between 10 preceding and current row", ValuePreceding(10), CurrentRow),
    +      ("between current row and 5 following", CurrentRow, ValueFollowing(5)),
    +      ("between 10 preceding and 5 following", ValuePreceding(10), ValueFollowing(5))
    +    )
    +    frameTypes.foreach {
    +      case (frameTypeSql, frameType) =>
    +        boundaries.foreach {
    +          case (boundarySql, begin, end) =>
    +            val query = s"foo(*) over (partition by a order by b $frameTypeSql $boundarySql)"
    +            val expr = windowed(Seq('a), Seq('b.asc), SpecifiedWindowFrame(frameType, begin, end))
    +            assertEqual(query, expr)
    +        }
    +    }
    +
    +    // We cannot use non integer constants.
    +    intercept("foo(*) over (partition by a order by b rows 10.0 preceding)",
    +      "Frame bound value must be a constant integer.")
    +
    +    // We cannot use an arbitrary expression.
    +    intercept("foo(*) over (partition by a order by b rows exp(b) preceding)",
    +      "Frame bound value must be a constant integer.")
    +
    +    // We cannot have a frame without an order by clause.
    +    intercept("foo(*) over (partition by a rows 10 preceding)", "mismatched input 'rows'")
    +  }
    +
    +  test("row constructor") {
    +    // Note that '(a)' will be interpreted as a nested expression.
    +    assertEqual("(a, b)", CreateStruct(Seq('a, 'b)))
    --- End diff --
    
    This is to support rows in virtual tables (inline tables), and I kinda like it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279440
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    +    assertEqual("foo(*)", 'foo.function(1))
    +    assertEqual("foo(a, b)", 'foo.function('a, 'b))
    +    assertEqual("foo(all a, b)", 'foo.function('a, 'b))
    +    assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b))
    +    assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b))
    +    assertEqual("`select`(all a, b)", 'select.function('a, 'b))
    +  }
    +
    +  test("window function expressions") {
    +    val func = 'foo.function(1)
    +    def windowed(
    +        partitioning: Seq[Expression] = Seq.empty,
    +        ordering: Seq[SortOrder] = Seq.empty,
    +        frame: WindowFrame = UnspecifiedFrame): Expression = {
    +      WindowExpression(func, WindowSpecDefinition(partitioning, ordering, frame))
    +    }
    +
    +    // Basic window testing.
    +    assertEqual("foo(*) over w1", UnresolvedWindowExpression(func, WindowSpecReference("w1")))
    +    assertEqual("foo(*) over ()", windowed())
    +    assertEqual("foo(*) over (partition by a, b)", windowed(Seq('a, 'b)))
    +    assertEqual("foo(*) over (order by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc )))
    +    assertEqual("foo(*) over (partition by a, b order by c)", windowed(Seq('a, 'b), Seq('c.asc)))
    +
    +    // Test use of expressions in window functions.
    +    assertEqual(
    +      "sum(product + 1) over (partition by ((product) + (1)) order by 2)",
    +      WindowExpression('sum.function('product + 1),
    +        WindowSpecDefinition(Seq('product + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
    +    assertEqual(
    +      "sum(product + 1) over (partition by ((product / 2) + 1) order by 2)",
    +      WindowExpression('sum.function('product + 1),
    +        WindowSpecDefinition(Seq('product / 2 + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
    +
    +    // Range/Row
    +    val frameTypes = Seq(("rows", RowFrame), ("range", RangeFrame))
    +    val boundaries = Seq(
    +      ("10 preceding", ValuePreceding(10), CurrentRow),
    +      ("3 + 1 following", ValueFollowing(4), CurrentRow), // Will fail during analysis
    +      ("unbounded preceding", UnboundedPreceding, CurrentRow),
    +      ("unbounded following", UnboundedFollowing, CurrentRow), // Will fail during analysis
    +      ("between unbounded preceding and current row", UnboundedPreceding, CurrentRow),
    +      ("between unbounded preceding and unbounded following",
    +        UnboundedPreceding, UnboundedFollowing),
    +      ("between 10 preceding and current row", ValuePreceding(10), CurrentRow),
    +      ("between current row and 5 following", CurrentRow, ValueFollowing(5)),
    +      ("between 10 preceding and 5 following", ValuePreceding(10), ValueFollowing(5))
    +    )
    +    frameTypes.foreach {
    +      case (frameTypeSql, frameType) =>
    +        boundaries.foreach {
    +          case (boundarySql, begin, end) =>
    +            val query = s"foo(*) over (partition by a order by b $frameTypeSql $boundarySql)"
    +            val expr = windowed(Seq('a), Seq('b.asc), SpecifiedWindowFrame(frameType, begin, end))
    +            assertEqual(query, expr)
    +        }
    +    }
    +
    +    // We cannot use non integer constants.
    +    intercept("foo(*) over (partition by a order by b rows 10.0 preceding)",
    +      "Frame bound value must be a constant integer.")
    +
    +    // We cannot use an arbitrary expression.
    +    intercept("foo(*) over (partition by a order by b rows exp(b) preceding)",
    +      "Frame bound value must be a constant integer.")
    +
    +    // We cannot have a frame without an order by clause.
    +    intercept("foo(*) over (partition by a rows 10 preceding)", "mismatched input 'rows'")
    +  }
    +
    +  test("row constructor") {
    +    // Note that '(a)' will be interpreted as a nested expression.
    +    assertEqual("(a, b)", CreateStruct(Seq('a, 'b)))
    +    assertEqual("(a, b, c)", CreateStruct(Seq('a, 'b, 'c)))
    +  }
    +
    +  test("scalar sub-query") {
    +    assertEqual(
    +      "(select max(val) from tbl) > current",
    +      ScalarSubquery(table("tbl").select('max.function('val))) > 'current)
    +    assertEqual(
    +      "a = (select b from s)",
    +      'a === ScalarSubquery(table("s").select('b)))
    +  }
    +
    +  test("case when") {
    +    assertEqual("case a when 1 then b when 2 then c else d end",
    +      CaseKeyWhen('a, Seq(1, 'b, 2, 'c, 'd)))
    +    assertEqual("case when a = 1 then b when a = 2 then c else d end",
    +      CaseWhen(Seq(('a === 1, 'b.expr), ('a === 2, 'c.expr)), 'd))
    +  }
    +
    +  test("dereference") {
    +    assertEqual("a.b", UnresolvedAttribute("a.b"))
    +    assertEqual("`select`.b", UnresolvedAttribute("select.b"))
    +    assertEqual("(a + b).b", ('a + 'b).getField("b")) // This will fail analysis.
    +    assertEqual("struct(a, b).b", 'struct.function('a, 'b).getField("b"))
    +  }
    +
    +  test("reference") {
    +    // Regular
    +    assertEqual("a", 'a)
    +
    +    // Starting with a digit.
    +    assertEqual("1a", Symbol("1a"))
    +
    +    // Quoted using a keyword.
    +    assertEqual("`select`", 'select)
    +
    +    // Unquoted using an unreserved keyword.
    +    assertEqual("columns", 'columns)
    +  }
    +
    +  test("subscript") {
    +    assertEqual("a[b]", 'a.getItem('b))
    +    assertEqual("a[1 + 1]", 'a.getItem(Literal(1) + 1))
    +    assertEqual("`c`.a[b]", UnresolvedAttribute("c.a").getItem('b))
    +  }
    +
    +  test("parenthesis") {
    +    assertEqual("(a)", 'a)
    +    assertEqual("r * (a + b)", 'r * ('a + 'b))
    +  }
    +
    +  test("type constructors") {
    +    // Dates.
    +    assertEqual("dAte '2016-03-11'", Literal(Date.valueOf("2016-03-11")))
    +    intercept[IllegalArgumentException] {
    +      parseExpression("DAtE 'mar 11 2016'")
    +    }
    +
    +    // Timestamps.
    +    assertEqual("tImEstAmp '2016-03-11 20:54:00.000'",
    +      Literal(Timestamp.valueOf("2016-03-11 20:54:00.000")))
    +    intercept[IllegalArgumentException] {
    +      parseExpression("timestamP '2016-33-11 20:54:00.000'")
    +    }
    +
    +    // Unsupported datatype.
    +    intercept("GEO '(10,-6)'", "Literals of type 'GEO' are currently not supported.")
    +  }
    +
    +  test("literals") {
    +    // NULL
    +    assertEqual("null", Literal(null))
    +
    +    // Boolean
    +    assertEqual("trUe", Literal(true))
    +    assertEqual("False", Literal(false))
    +
    +    // Integral should have the narrowest possible type
    +    assertEqual("787324", Literal(787324))
    +    assertEqual("7873247234798249234", Literal(7873247234798249234L))
    +    assertEqual("78732472347982492793712334",
    +      Literal(BigDecimal("78732472347982492793712334").underlying()))
    +
    +    // Decimal
    +    assertEqual("7873247234798249279371.2334",
    +      Literal(BigDecimal("7873247234798249279371.2334").underlying()))
    +
    +    // Scientific Decimal
    +    assertEqual("9.0e1", 90d)
    +    assertEqual(".9e+2", 90d)
    +    assertEqual("0.9e+2", 90d)
    +    assertEqual("900e-1", 90d)
    +    assertEqual("900.0E-1", 90d)
    +    assertEqual("9.e+1", 90d)
    +    intercept(".e3")
    +
    +    // Tiny Int Literal
    +    assertEqual("10Y", Literal(10.toByte))
    +    intercept("-1000Y")
    +
    +    // Small Int Literal
    +    assertEqual("10S", Literal(10.toShort))
    +    intercept("40000S")
    +
    +    // Long Int Literal
    +    assertEqual("10L", Literal(10L))
    +    intercept("78732472347982492793712334L")
    +
    +    // Double Literal
    +    assertEqual("10.0D", Literal(10.0D))
    +    // TODO we need to figure out if we should throw an exception here!
    +    assertEqual("1E309", Literal(Double.PositiveInfinity))
    +  }
    +
    +  test("strings") {
    +    // Single Strings.
    +    assertEqual("\"hello\"", "hello")
    +    assertEqual("'hello'", "hello")
    +
    +    // Multi-Strings.
    +    assertEqual("\"hello\" 'world'", "helloworld")
    +    assertEqual("'hello' \" \" 'world'", "hello world")
    +
    +    // 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a
    +    // regular '%'; to get the correct result you need to add another escaped '\'.
    +    // TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method?
    +    assertEqual("'pattern%'", "pattern%")
    +    assertEqual("'no-pattern\\%'", "no-pattern\\%")
    +    assertEqual("'pattern\\\\%'", "pattern\\%")
    +    assertEqual("'pattern\\\\\\%'", "pattern\\\\%")
    +
    +    // Escaped characters.
    +    // See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html
    +    assertEqual("'\\0'", "\u0000") // ASCII NUL (X'00')
    +    assertEqual("'\\''", "\'")     // Single quote
    +    assertEqual("'\\\"'", "\"")    // Double quote
    +    assertEqual("'\\b'", "\b")     // Backspace
    +    assertEqual("'\\n'", "\n")     // Newline
    +    assertEqual("'\\r'", "\r")     // Carriage return
    +    assertEqual("'\\t'", "\t")     // Tab character
    +    assertEqual("'\\Z'", "\u001A") // ASCII 26 - CTRL + Z (EOF on windows)
    +
    +    // Octals
    +    assertEqual("'\\110\\145\\154\\154\\157\\041'", "Hello!")
    +
    +    // Unicode
    +    assertEqual("'\\u0087\\u0111\\u0114\\u0108\\u0100\\u0032\\u0058\\u0041'", "World :)")
    +  }
    +
    +  test("intervals") {
    +    def intervalLiteral(u: String, s: String): Literal = {
    +      Literal(CalendarInterval.fromSingleUnitString(u, s))
    +    }
    +
    +    // Empty interval statement
    +    intercept("interval", "at least one time unit should be given for interval literal")
    +
    +    // Single Intervals.
    +    val units = Seq(
    +      "year",
    +      "month",
    +      "week",
    +      "day",
    +      "hour",
    +      "minute",
    +      "second",
    +      "millisecond",
    +      "microsecond")
    +    val forms = Seq("", "s")
    +    val values = Seq("0", "10", "-7", "21")
    +    units.foreach { unit =>
    +      forms.foreach { form =>
    +         values.foreach { value =>
    +           val expected = intervalLiteral(unit, value)
    +           assertEqual(s"interval $value $unit$form", expected)
    +           assertEqual(s"interval '$value' $unit$form", expected)
    +         }
    +      }
    +    }
    +
    +    // Hive nanosecond notation.
    +    assertEqual("interval 13.123456789 seconds", intervalLiteral("second", "13.123456789"))
    +    assertEqual("interval -13.123456789 second", intervalLiteral("second", "-13.123456789"))
    +
    +    // Non Existing unit
    +    intercept("interval 10 nanoseconds", "No interval can be constructed")
    +
    +    // Year-Month intervals.
    +    val yearMonthValues = Seq("123-10", "496-0", "-2-3", "-123-0")
    +    yearMonthValues.foreach { value =>
    +      val result = Literal(CalendarInterval.fromYearMonthString(value))
    +      assertEqual(s"interval '$value' year to month", result)
    +    }
    +
    +    // Year-Month intervals.
    --- End diff --
    
    We do after line 463. I added the wrong comment.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193656062
  
    **[Test build #52646 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52646/consoleFull)** for PR 11557 at commit [`2c9a092`](https://github.com/apache/spark/commit/2c9a092e2c3c944ecfeaa9dbdd55fd2665fb3d23).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198134876
  
    **[Test build #53480 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53480/consoleFull)** for PR 11557 at commit [`b87f2b8`](https://github.com/apache/spark/commit/b87f2b80e770f311326eeefb23e9d9b6ced62aa3).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195882238
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53020/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-200527025
  
    @cloud-fan I have created a gist of the generated files: https://gist.github.com/hvanhovell/2ee2149efd6cab79e880
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-200518822
  
    **[Test build #53964 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53964/consoleFull)** for PR 11557 at commit [`be6f5c8`](https://github.com/apache/spark/commit/be6f5c82956152ac33c78d484ee5fe8952e18632).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193183241
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/52550/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195885833
  
    **[Test build #53022 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53022/consoleFull)** for PR 11557 at commit [`460ef0d`](https://github.com/apache/spark/commit/460ef0d77a366602c89d87e9518fc94a8b721dd3).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57271566
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    +    assertEqual("foo(*)", 'foo.function(1))
    +    assertEqual("foo(a, b)", 'foo.function('a, 'b))
    +    assertEqual("foo(all a, b)", 'foo.function('a, 'b))
    +    assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b))
    +    assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b))
    +    assertEqual("`select`(all a, b)", 'select.function('a, 'b))
    +  }
    +
    +  test("window function expressions") {
    +    val func = 'foo.function(1)
    +    def windowed(
    +        partitioning: Seq[Expression] = Seq.empty,
    +        ordering: Seq[SortOrder] = Seq.empty,
    +        frame: WindowFrame = UnspecifiedFrame): Expression = {
    +      WindowExpression(func, WindowSpecDefinition(partitioning, ordering, frame))
    +    }
    +
    +    // Basic window testing.
    +    assertEqual("foo(*) over w1", UnresolvedWindowExpression(func, WindowSpecReference("w1")))
    +    assertEqual("foo(*) over ()", windowed())
    +    assertEqual("foo(*) over (partition by a, b)", windowed(Seq('a, 'b)))
    +    assertEqual("foo(*) over (order by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc )))
    +    assertEqual("foo(*) over (partition by a, b order by c)", windowed(Seq('a, 'b), Seq('c.asc)))
    +
    +    // Test use of expressions in window functions.
    +    assertEqual(
    +      "sum(product + 1) over (partition by ((product) + (1)) order by 2)",
    +      WindowExpression('sum.function('product + 1),
    +        WindowSpecDefinition(Seq('product + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
    +    assertEqual(
    +      "sum(product + 1) over (partition by ((product / 2) + 1) order by 2)",
    +      WindowExpression('sum.function('product + 1),
    +        WindowSpecDefinition(Seq('product / 2 + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
    +
    +    // Range/Row
    +    val frameTypes = Seq(("rows", RowFrame), ("range", RangeFrame))
    +    val boundaries = Seq(
    +      ("10 preceding", ValuePreceding(10), CurrentRow),
    +      ("3 + 1 following", ValueFollowing(4), CurrentRow), // Will fail during analysis
    +      ("unbounded preceding", UnboundedPreceding, CurrentRow),
    +      ("unbounded following", UnboundedFollowing, CurrentRow), // Will fail during analysis
    +      ("between unbounded preceding and current row", UnboundedPreceding, CurrentRow),
    +      ("between unbounded preceding and unbounded following",
    +        UnboundedPreceding, UnboundedFollowing),
    +      ("between 10 preceding and current row", ValuePreceding(10), CurrentRow),
    +      ("between current row and 5 following", CurrentRow, ValueFollowing(5)),
    +      ("between 10 preceding and 5 following", ValuePreceding(10), ValueFollowing(5))
    +    )
    +    frameTypes.foreach {
    +      case (frameTypeSql, frameType) =>
    +        boundaries.foreach {
    +          case (boundarySql, begin, end) =>
    +            val query = s"foo(*) over (partition by a order by b $frameTypeSql $boundarySql)"
    +            val expr = windowed(Seq('a), Seq('b.asc), SpecifiedWindowFrame(frameType, begin, end))
    +            assertEqual(query, expr)
    +        }
    +    }
    +
    +    // We cannot use non integer constants.
    +    intercept("foo(*) over (partition by a order by b rows 10.0 preceding)",
    +      "Frame bound value must be a constant integer.")
    +
    +    // We cannot use an arbitrary expression.
    +    intercept("foo(*) over (partition by a order by b rows exp(b) preceding)",
    +      "Frame bound value must be a constant integer.")
    +
    +    // We cannot have a frame without an order by clause.
    +    intercept("foo(*) over (partition by a rows 10 preceding)", "mismatched input 'rows'")
    +  }
    +
    +  test("row constructor") {
    +    // Note that '(a)' will be interpreted as a nested expression.
    +    assertEqual("(a, b)", CreateStruct(Seq('a, 'b)))
    --- End diff --
    
    is it standard? Previously we only `struct(a, b)` as row constructor


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57271935
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    +    assertEqual("foo(*)", 'foo.function(1))
    +    assertEqual("foo(a, b)", 'foo.function('a, 'b))
    +    assertEqual("foo(all a, b)", 'foo.function('a, 'b))
    +    assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b))
    +    assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b))
    +    assertEqual("`select`(all a, b)", 'select.function('a, 'b))
    +  }
    +
    +  test("window function expressions") {
    +    val func = 'foo.function(1)
    +    def windowed(
    +        partitioning: Seq[Expression] = Seq.empty,
    +        ordering: Seq[SortOrder] = Seq.empty,
    +        frame: WindowFrame = UnspecifiedFrame): Expression = {
    +      WindowExpression(func, WindowSpecDefinition(partitioning, ordering, frame))
    +    }
    +
    +    // Basic window testing.
    +    assertEqual("foo(*) over w1", UnresolvedWindowExpression(func, WindowSpecReference("w1")))
    +    assertEqual("foo(*) over ()", windowed())
    +    assertEqual("foo(*) over (partition by a, b)", windowed(Seq('a, 'b)))
    +    assertEqual("foo(*) over (order by a desc, b asc)", windowed(Seq.empty, Seq('a.desc, 'b.asc )))
    +    assertEqual("foo(*) over (partition by a, b order by c)", windowed(Seq('a, 'b), Seq('c.asc)))
    +
    +    // Test use of expressions in window functions.
    +    assertEqual(
    +      "sum(product + 1) over (partition by ((product) + (1)) order by 2)",
    +      WindowExpression('sum.function('product + 1),
    +        WindowSpecDefinition(Seq('product + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
    +    assertEqual(
    +      "sum(product + 1) over (partition by ((product / 2) + 1) order by 2)",
    +      WindowExpression('sum.function('product + 1),
    +        WindowSpecDefinition(Seq('product / 2 + 1), Seq(Literal(2).asc), UnspecifiedFrame)))
    +
    +    // Range/Row
    +    val frameTypes = Seq(("rows", RowFrame), ("range", RangeFrame))
    +    val boundaries = Seq(
    +      ("10 preceding", ValuePreceding(10), CurrentRow),
    +      ("3 + 1 following", ValueFollowing(4), CurrentRow), // Will fail during analysis
    +      ("unbounded preceding", UnboundedPreceding, CurrentRow),
    +      ("unbounded following", UnboundedFollowing, CurrentRow), // Will fail during analysis
    +      ("between unbounded preceding and current row", UnboundedPreceding, CurrentRow),
    +      ("between unbounded preceding and unbounded following",
    +        UnboundedPreceding, UnboundedFollowing),
    +      ("between 10 preceding and current row", ValuePreceding(10), CurrentRow),
    +      ("between current row and 5 following", CurrentRow, ValueFollowing(5)),
    +      ("between 10 preceding and 5 following", ValuePreceding(10), ValueFollowing(5))
    +    )
    +    frameTypes.foreach {
    +      case (frameTypeSql, frameType) =>
    +        boundaries.foreach {
    +          case (boundarySql, begin, end) =>
    +            val query = s"foo(*) over (partition by a order by b $frameTypeSql $boundarySql)"
    +            val expr = windowed(Seq('a), Seq('b.asc), SpecifiedWindowFrame(frameType, begin, end))
    +            assertEqual(query, expr)
    +        }
    +    }
    +
    +    // We cannot use non integer constants.
    +    intercept("foo(*) over (partition by a order by b rows 10.0 preceding)",
    +      "Frame bound value must be a constant integer.")
    +
    +    // We cannot use an arbitrary expression.
    +    intercept("foo(*) over (partition by a order by b rows exp(b) preceding)",
    +      "Frame bound value must be a constant integer.")
    +
    +    // We cannot have a frame without an order by clause.
    +    intercept("foo(*) over (partition by a rows 10 preceding)", "mismatched input 'rows'")
    +  }
    +
    +  test("row constructor") {
    +    // Note that '(a)' will be interpreted as a nested expression.
    +    assertEqual("(a, b)", CreateStruct(Seq('a, 'b)))
    +    assertEqual("(a, b, c)", CreateStruct(Seq('a, 'b, 'c)))
    +  }
    +
    +  test("scalar sub-query") {
    +    assertEqual(
    +      "(select max(val) from tbl) > current",
    +      ScalarSubquery(table("tbl").select('max.function('val))) > 'current)
    +    assertEqual(
    +      "a = (select b from s)",
    +      'a === ScalarSubquery(table("s").select('b)))
    +  }
    +
    +  test("case when") {
    +    assertEqual("case a when 1 then b when 2 then c else d end",
    +      CaseKeyWhen('a, Seq(1, 'b, 2, 'c, 'd)))
    +    assertEqual("case when a = 1 then b when a = 2 then c else d end",
    +      CaseWhen(Seq(('a === 1, 'b.expr), ('a === 2, 'c.expr)), 'd))
    +  }
    +
    +  test("dereference") {
    +    assertEqual("a.b", UnresolvedAttribute("a.b"))
    +    assertEqual("`select`.b", UnresolvedAttribute("select.b"))
    +    assertEqual("(a + b).b", ('a + 'b).getField("b")) // This will fail analysis.
    +    assertEqual("struct(a, b).b", 'struct.function('a, 'b).getField("b"))
    +  }
    +
    +  test("reference") {
    +    // Regular
    +    assertEqual("a", 'a)
    +
    +    // Starting with a digit.
    +    assertEqual("1a", Symbol("1a"))
    +
    +    // Quoted using a keyword.
    +    assertEqual("`select`", 'select)
    +
    +    // Unquoted using an unreserved keyword.
    +    assertEqual("columns", 'columns)
    +  }
    +
    +  test("subscript") {
    +    assertEqual("a[b]", 'a.getItem('b))
    +    assertEqual("a[1 + 1]", 'a.getItem(Literal(1) + 1))
    +    assertEqual("`c`.a[b]", UnresolvedAttribute("c.a").getItem('b))
    +  }
    +
    +  test("parenthesis") {
    +    assertEqual("(a)", 'a)
    +    assertEqual("r * (a + b)", 'r * ('a + 'b))
    +  }
    +
    +  test("type constructors") {
    +    // Dates.
    +    assertEqual("dAte '2016-03-11'", Literal(Date.valueOf("2016-03-11")))
    +    intercept[IllegalArgumentException] {
    +      parseExpression("DAtE 'mar 11 2016'")
    +    }
    +
    +    // Timestamps.
    +    assertEqual("tImEstAmp '2016-03-11 20:54:00.000'",
    +      Literal(Timestamp.valueOf("2016-03-11 20:54:00.000")))
    +    intercept[IllegalArgumentException] {
    +      parseExpression("timestamP '2016-33-11 20:54:00.000'")
    +    }
    +
    +    // Unsupported datatype.
    +    intercept("GEO '(10,-6)'", "Literals of type 'GEO' are currently not supported.")
    +  }
    +
    +  test("literals") {
    +    // NULL
    +    assertEqual("null", Literal(null))
    +
    +    // Boolean
    +    assertEqual("trUe", Literal(true))
    +    assertEqual("False", Literal(false))
    +
    +    // Integral should have the narrowest possible type
    +    assertEqual("787324", Literal(787324))
    +    assertEqual("7873247234798249234", Literal(7873247234798249234L))
    +    assertEqual("78732472347982492793712334",
    +      Literal(BigDecimal("78732472347982492793712334").underlying()))
    +
    +    // Decimal
    +    assertEqual("7873247234798249279371.2334",
    +      Literal(BigDecimal("7873247234798249279371.2334").underlying()))
    +
    +    // Scientific Decimal
    +    assertEqual("9.0e1", 90d)
    +    assertEqual(".9e+2", 90d)
    +    assertEqual("0.9e+2", 90d)
    +    assertEqual("900e-1", 90d)
    +    assertEqual("900.0E-1", 90d)
    +    assertEqual("9.e+1", 90d)
    +    intercept(".e3")
    +
    +    // Tiny Int Literal
    +    assertEqual("10Y", Literal(10.toByte))
    +    intercept("-1000Y")
    +
    +    // Small Int Literal
    +    assertEqual("10S", Literal(10.toShort))
    +    intercept("40000S")
    +
    +    // Long Int Literal
    +    assertEqual("10L", Literal(10L))
    +    intercept("78732472347982492793712334L")
    +
    +    // Double Literal
    +    assertEqual("10.0D", Literal(10.0D))
    +    // TODO we need to figure out if we should throw an exception here!
    +    assertEqual("1E309", Literal(Double.PositiveInfinity))
    +  }
    +
    +  test("strings") {
    +    // Single Strings.
    +    assertEqual("\"hello\"", "hello")
    +    assertEqual("'hello'", "hello")
    +
    +    // Multi-Strings.
    +    assertEqual("\"hello\" 'world'", "helloworld")
    +    assertEqual("'hello' \" \" 'world'", "hello world")
    +
    +    // 'LIKE' string literals. Notice that an escaped '%' is the same as an escaped '\' and a
    +    // regular '%'; to get the correct result you need to add another escaped '\'.
    +    // TODO figure out if we shouldn't change the ParseUtils.unescapeSQLString method?
    +    assertEqual("'pattern%'", "pattern%")
    +    assertEqual("'no-pattern\\%'", "no-pattern\\%")
    +    assertEqual("'pattern\\\\%'", "pattern\\%")
    +    assertEqual("'pattern\\\\\\%'", "pattern\\\\%")
    +
    +    // Escaped characters.
    +    // See: http://dev.mysql.com/doc/refman/5.7/en/string-literals.html
    +    assertEqual("'\\0'", "\u0000") // ASCII NUL (X'00')
    +    assertEqual("'\\''", "\'")     // Single quote
    +    assertEqual("'\\\"'", "\"")    // Double quote
    +    assertEqual("'\\b'", "\b")     // Backspace
    +    assertEqual("'\\n'", "\n")     // Newline
    +    assertEqual("'\\r'", "\r")     // Carriage return
    +    assertEqual("'\\t'", "\t")     // Tab character
    +    assertEqual("'\\Z'", "\u001A") // ASCII 26 - CTRL + Z (EOF on windows)
    +
    +    // Octals
    +    assertEqual("'\\110\\145\\154\\154\\157\\041'", "Hello!")
    +
    +    // Unicode
    +    assertEqual("'\\u0087\\u0111\\u0114\\u0108\\u0100\\u0032\\u0058\\u0041'", "World :)")
    +  }
    +
    +  test("intervals") {
    +    def intervalLiteral(u: String, s: String): Literal = {
    +      Literal(CalendarInterval.fromSingleUnitString(u, s))
    +    }
    +
    +    // Empty interval statement
    +    intercept("interval", "at least one time unit should be given for interval literal")
    +
    +    // Single Intervals.
    +    val units = Seq(
    +      "year",
    +      "month",
    +      "week",
    +      "day",
    +      "hour",
    +      "minute",
    +      "second",
    +      "millisecond",
    +      "microsecond")
    +    val forms = Seq("", "s")
    +    val values = Seq("0", "10", "-7", "21")
    +    units.foreach { unit =>
    +      forms.foreach { form =>
    +         values.foreach { value =>
    +           val expected = intervalLiteral(unit, value)
    +           assertEqual(s"interval $value $unit$form", expected)
    +           assertEqual(s"interval '$value' $unit$form", expected)
    +         }
    +      }
    +    }
    +
    +    // Hive nanosecond notation.
    +    assertEqual("interval 13.123456789 seconds", intervalLiteral("second", "13.123456789"))
    +    assertEqual("interval -13.123456789 second", intervalLiteral("second", "-13.123456789"))
    +
    +    // Non Existing unit
    +    intercept("interval 10 nanoseconds", "No interval can be constructed")
    +
    +    // Year-Month intervals.
    +    val yearMonthValues = Seq("123-10", "496-0", "-2-3", "-123-0")
    +    yearMonthValues.foreach { value =>
    +      val result = Literal(CalendarInterval.fromYearMonthString(value))
    +      assertEqual(s"interval '$value' year to month", result)
    +    }
    +
    +    // Year-Month intervals.
    --- End diff --
    
    how about the day-time intervals?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195911930
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53022/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #11557: [SPARK-13713][SQL] Migrate parser from ANTLR3 to ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r73651843
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1452 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import ParserUtils._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
    +    visit(ctx.dataType).asInstanceOf[DataType]
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Make sure we do not try to create a plan for a native command.
    +   */
    +  override def visitExecuteNativeCommand(ctx: ExecuteNativeCommandContext): LogicalPlan = null
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          throw new ParseException("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(string(pattern)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }
    +
    +      // Check for duplicate names.
    +      ctes.groupBy(_._1).filter(_._2.size > 1).foreach {
    +        case (name, _) =>
    +          throw new ParseException(
    +            s"Name '$name' is used for multiple common table expressions", ctx)
    +      }
    +
    +      With(query, ctes.toMap)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryResultClauses).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryResultClauses).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Create a partition specification map.
    +   */
    +  override def visitPartitionSpec(
    +      ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) {
    +    ctx.partitionVal.asScala.map { pVal =>
    +      val name = pVal.identifier.getText.toLowerCase
    +      val value = Option(pVal.constant).map(visitStringConstant)
    +      name -> value
    +    }.toMap
    +  }
    +
    +  /**
    +   * Create a partition specification map without optional values.
    +   */
    +  protected def visitNonOptionalPartitionSpec(
    +      ctx: PartitionSpecContext): Map[String, String] = withOrigin(ctx) {
    +    visitPartitionSpec(ctx).mapValues(_.orNull).map(identity)
    +  }
    +
    +  /**
    +   * Convert a constant of any type into a string. This is typically used in DDL commands, and its
    +   * main purpose is to prevent slight differences due to back to back conversions i.e.:
    +   * String -> Literal -> String.
    +   */
    +  protected def visitStringConstant(ctx: ConstantContext): String = withOrigin(ctx) {
    +    ctx match {
    +      case s: StringLiteralContext => createString(s)
    +      case o => o.getText
    +    }
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan. These
    +   * clauses determine the shape (ordering/partitioning/rows) of the query result.
    +   */
    +  private def withQueryResultClauses(
    +      ctx: QueryOrganizationContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
    +    val withOrder = if (
    +      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // ORDER BY ...
    +      Sort(order.asScala.map(visitSortItem), global = true, query)
    +    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ...
    +      Sort(sort.asScala.map(visitSortItem), global = false, query)
    +    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // DISTRIBUTE BY ...
    +      RepartitionByExpression(expressionList(distributeBy), query)
    +    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ... DISTRIBUTE BY ...
    +      Sort(
    +        sort.asScala.map(visitSortItem),
    +        global = false,
    +        RepartitionByExpression(expressionList(distributeBy), query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
    +      // CLUSTER BY ...
    +      val expressions = expressionList(clusterBy)
    +      Sort(
    +        expressions.map(SortOrder(_, Ascending)),
    +        global = false,
    +        RepartitionByExpression(expressions, query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // [EMPTY]
    +      query
    +    } else {
    +      throw new ParseException(
    +        "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
    +    }
    +
    +    // WINDOWS
    +    val withWindow = withOrder.optionalMap(windows)(withWindows)
    +
    +    // LIMIT
    +    withWindow.optional(limit) {
    +      Limit(typedVisit(limit), withWindow)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan using a query specification.
    +   */
    +  override def visitQuerySpecification(
    +      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
    +    val from = OneRowRelation.optional(ctx.fromClause) {
    +      visitFromClause(ctx.fromClause)
    +    }
    +    withQuerySpecification(ctx, from)
    +  }
    +
    +  /**
    +   * Add a query specification to a logical plan. The query specification is the core of the logical
    +   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
    +   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
    +   *
    +   * Note that query hints are ignored (both by the parser and the builder).
    +   */
    +  private def withQuerySpecification(
    +      ctx: QuerySpecificationContext,
    +      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // WHERE
    +    def filter(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = {
    +      Filter(expression(ctx), plan)
    +    }
    +
    +    // Expressions.
    +    val expressions = Option(namedExpressionSeq).toSeq
    +      .flatMap(_.namedExpression.asScala)
    +      .map(typedVisit[Expression])
    +
    +    // Create either a transform or a regular query.
    +    val specType = Option(kind).map(_.getType).getOrElse(SqlBaseParser.SELECT)
    +    specType match {
    +      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
    +        // Transform
    +
    +        // Add where.
    +        val withFilter = relation.optionalMap(where)(filter)
    +
    +        // Create the attributes.
    +        val (attributes, schemaLess) = if (colTypeList != null) {
    +          // Typed return columns.
    +          (createStructType(colTypeList).toAttributes, false)
    +        } else if (identifierSeq != null) {
    +          // Untyped return columns.
    +          val attrs = visitIdentifierSeq(identifierSeq).map { name =>
    +            AttributeReference(name, StringType, nullable = true)()
    +          }
    +          (attrs, false)
    +        } else {
    +          (Seq(AttributeReference("key", StringType)(),
    +            AttributeReference("value", StringType)()), true)
    +        }
    +
    +        // Create the transform.
    +        ScriptTransformation(
    +          expressions,
    +          string(script),
    +          attributes,
    +          withFilter,
    +          withScriptIOSchema(inRowFormat, recordWriter, outRowFormat, recordReader, schemaLess))
    +
    +      case SqlBaseParser.SELECT =>
    +        // Regular select
    +
    +        // Add lateral views.
    +        val withLateralView = ctx.lateralView.asScala.foldLeft(relation)(withGenerate)
    +
    +        // Add where.
    +        val withFilter = withLateralView.optionalMap(where)(filter)
    +
    +        // Add aggregation or a project.
    +        val namedExpressions = expressions.map {
    +          case e: NamedExpression => e
    +          case e: Expression => UnresolvedAlias(e)
    +        }
    +        val withProject = if (aggregation != null) {
    +          withAggregation(aggregation, namedExpressions, withFilter)
    +        } else if (namedExpressions.nonEmpty) {
    +          Project(namedExpressions, withFilter)
    +        } else {
    +          withFilter
    +        }
    +
    +        // Having
    +        val withHaving = withProject.optional(having) {
    +          // Note that we added a cast to boolean. If the expression itself is already boolean,
    +          // the optimizer will get rid of the unnecessary cast.
    +          Filter(Cast(expression(having), BooleanType), withProject)
    +        }
    +
    +        // Distinct
    +        val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) {
    +          Distinct(withHaving)
    +        } else {
    +          withHaving
    +        }
    +
    +        // Window
    +        withDistinct.optionalMap(windows)(withWindows)
    +    }
    +  }
    +
    +  /**
    +   * Create a (Hive based) [[ScriptInputOutputSchema]].
    +   */
    +  protected def withScriptIOSchema(
    +      inRowFormat: RowFormatContext,
    +      recordWriter: Token,
    +      outRowFormat: RowFormatContext,
    +      recordReader: Token,
    +      schemaLess: Boolean): ScriptInputOutputSchema = null
    +
    +  /**
    +   * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
    +   * separated) relations here, these get converted into a single plan by condition-less inner join.
    +   */
    +  override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
    +    val from = ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None))
    +    ctx.lateralView.asScala.foldLeft(from)(withGenerate)
    +  }
    +
    +  /**
    +   * Connect two queries by a Set operator.
    +   *
    +   * Supported Set operators are:
    +   * - UNION [DISTINCT]
    +   * - UNION ALL
    +   * - EXCEPT [DISTINCT]
    +   * - INTERSECT [DISTINCT]
    +   */
    +  override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
    +    val left = plan(ctx.left)
    +    val right = plan(ctx.right)
    +    val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.UNION if all =>
    +        Union(left, right)
    +      case SqlBaseParser.UNION =>
    +        Distinct(Union(left, right))
    +      case SqlBaseParser.INTERSECT if all =>
    +        throw new ParseException("INTERSECT ALL is not supported.", ctx)
    +      case SqlBaseParser.INTERSECT =>
    +        Intersect(left, right)
    +      case SqlBaseParser.EXCEPT if all =>
    +        throw new ParseException("EXCEPT ALL is not supported.", ctx)
    +      case SqlBaseParser.EXCEPT =>
    +        Except(left, right)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[WithWindowDefinition]] operator to a logical plan.
    +   */
    +  private def withWindows(
    +      ctx: WindowsContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Collect all window specifications defined in the WINDOW clause.
    +    val baseWindowMap = ctx.namedWindow.asScala.map {
    +      wCtx =>
    +        (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
    +    }.toMap
    +
    +    // Handle cases like
    +    // window w1 as (partition by p_mfgr order by p_name
    +    //               range between 2 preceding and 2 following),
    +    //        w2 as w1
    +    val windowMapView = baseWindowMap.mapValues {
    +      case WindowSpecReference(name) =>
    +        baseWindowMap.get(name) match {
    +          case Some(spec: WindowSpecDefinition) =>
    +            spec
    +          case Some(ref) =>
    +            throw new ParseException(s"Window reference '$name' is not a window specification", ctx)
    +          case None =>
    +            throw new ParseException(s"Cannot resolve window reference '$name'", ctx)
    +        }
    +      case spec: WindowSpecDefinition => spec
    +    }
    +
    +    // Note that mapValues creates a view instead of materialized map. We force materialization by
    +    // mapping over identity.
    +    WithWindowDefinition(windowMapView.map(identity), query)
    +  }
    +
    +  /**
    +   * Add an [[Aggregate]] to a logical plan.
    +   */
    +  private def withAggregation(
    +      ctx: AggregationContext,
    +      selectExpressions: Seq[NamedExpression],
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    val groupByExpressions = expressionList(groupingExpressions)
    +
    +    if (GROUPING != null) {
    +      // GROUP BY .... GROUPING SETS (...)
    +      val expressionMap = groupByExpressions.zipWithIndex.toMap
    +      val numExpressions = expressionMap.size
    +      val mask = (1 << numExpressions) - 1
    +      val masks = ctx.groupingSet.asScala.map {
    +        _.expression.asScala.foldLeft(mask) {
    +          case (bitmap, eCtx) =>
    +            // Find the index of the expression.
    +            val e = typedVisit[Expression](eCtx)
    +            val index = expressionMap.find(_._1.semanticEquals(e)).map(_._2).getOrElse(
    +              throw new ParseException(
    +                s"$e doesn't show up in the GROUP BY list", ctx))
    +            // 0 means that the column at the given index is a grouping column, 1 means it is not,
    +            // so we unset the bit in bitmap.
    +            bitmap & ~(1 << (numExpressions - 1 - index))
    +        }
    +      }
    +      GroupingSets(masks, groupByExpressions, query, selectExpressions)
    +    } else {
    +      // GROUP BY .... (WITH CUBE | WITH ROLLUP)?
    +      val mappedGroupByExpressions = if (CUBE != null) {
    +        Seq(Cube(groupByExpressions))
    +      } else if (ROLLUP != null) {
    +        Seq(Rollup(groupByExpressions))
    +      } else {
    +        groupByExpressions
    +      }
    +      Aggregate(mappedGroupByExpressions, selectExpressions, query)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[Generate]] (Lateral View) to a logical plan.
    +   */
    +  private def withGenerate(
    +      query: LogicalPlan,
    +      ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
    +    val expressions = expressionList(ctx.expression)
    +
    +    // Create the generator.
    +    val generator = ctx.qualifiedName.getText.toLowerCase match {
    +      case "explode" if expressions.size == 1 =>
    +        Explode(expressions.head)
    +      case "json_tuple" =>
    +        JsonTuple(expressions)
    +      case other =>
    +        withGenerator(other, expressions, ctx)
    +    }
    +
    +    Generate(
    +      generator,
    +      join = true,
    +      outer = ctx.OUTER != null,
    +      Some(ctx.tblName.getText.toLowerCase),
    +      ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply),
    +      query)
    +  }
    +
    +  /**
    +   * Create a [[Generator]]. Override this method in order to support custom Generators.
    +   */
    +  protected def withGenerator(
    +      name: String,
    +      expressions: Seq[Expression],
    +      ctx: LateralViewContext): Generator = {
    +    throw new ParseException(s"Generator function '$name' is not supported", ctx)
    +  }
    +
    +  /**
    +   * Create a joins between two or more logical plans.
    +   */
    +  override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) {
    +    /** Build a join between two plans. */
    +    def join(ctx: JoinRelationContext, left: LogicalPlan, right: LogicalPlan): Join = {
    +      val baseJoinType = ctx.joinType match {
    +        case null => Inner
    +        case jt if jt.FULL != null => FullOuter
    +        case jt if jt.SEMI != null => LeftSemi
    +        case jt if jt.LEFT != null => LeftOuter
    +        case jt if jt.RIGHT != null => RightOuter
    +        case _ => Inner
    +      }
    +
    +      // Resolve the join type and join condition
    +      val (joinType, condition) = Option(ctx.joinCriteria) match {
    +        case Some(c) if c.USING != null =>
    +          val columns = c.identifier.asScala.map { column =>
    +            UnresolvedAttribute.quoted(column.getText)
    +          }
    +          (UsingJoin(baseJoinType, columns), None)
    +        case Some(c) if c.booleanExpression != null =>
    +          (baseJoinType, Option(expression(c.booleanExpression)))
    +        case None if ctx.NATURAL != null =>
    +          (NaturalJoin(baseJoinType), None)
    +        case None =>
    +          (baseJoinType, None)
    +      }
    +      Join(left, right, joinType, condition)
    +    }
    +
    +    // Handle all consecutive join clauses. ANTLR produces a right nested tree in which the the
    +    // first join clause is at the top. However fields of previously referenced tables can be used
    +    // in following join clauses. The tree needs to be reversed in order to make this work.
    +    var result = plan(ctx.left)
    +    var current = ctx
    +    while (current != null) {
    +      current.right match {
    +        case right: JoinRelationContext =>
    +          result = join(current, result, plan(right.left))
    +          current = right
    +        case right =>
    +          result = join(current, result, plan(right))
    +          current = null
    +      }
    +    }
    +    result
    +  }
    +
    +  /**
    +   * Add a [[Sample]] to a logical plan.
    +   *
    +   * This currently supports the following sampling methods:
    +   * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
    +   * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
    +   * are defined as a number between 0 and 100.
    +   * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
    +   */
    +  private def withSample(ctx: SampleContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Create a sampled plan if we need one.
    +    def sample(fraction: Double): Sample = {
    +      // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
    +      // function takes X PERCENT as the input and the range of X is [0, 100], we need to
    +      // adjust the fraction.
    +      val eps = RandomSampler.roundingEpsilon
    +      assert(fraction >= 0.0 - eps && fraction <= 1.0 + eps,
    +        s"Sampling fraction ($fraction) must be on interval [0, 1]",
    +        ctx)
    +      Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)(true)
    +    }
    +
    +    ctx.sampleType.getType match {
    +      case SqlBaseParser.ROWS =>
    +        Limit(expression(ctx.expression), query)
    +
    +      case SqlBaseParser.PERCENTLIT =>
    +        val fraction = ctx.percentage.getText.toDouble
    +        sample(fraction / 100.0d)
    +
    +      case SqlBaseParser.BUCKET if ctx.ON != null =>
    +        throw new ParseException("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx)
    +
    +      case SqlBaseParser.BUCKET =>
    +        sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a sub-query.
    +   */
    +  override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryNoWith)
    +  }
    +
    +  /**
    +   * Create an un-aliased table reference. This is typically used for top-level table references,
    +   * for example:
    +   * {{{
    +   *   INSERT INTO db.tbl2
    +   *   TABLE db.tbl1
    +   * }}}
    +   */
    +  override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None)
    +  }
    +
    +  /**
    +   * Create an aliased table reference. This is typically used in FROM clauses.
    +   */
    +  override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
    +    val table = UnresolvedRelation(
    +      visitTableIdentifier(ctx.tableIdentifier),
    +      Option(ctx.identifier).map(_.getText))
    +    table.optionalMap(ctx.sample)(withSample)
    +  }
    +
    +  /**
    +   * Create an inline table (a virtual table in Hive parlance).
    +   */
    +  override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
    +    // Get the backing expressions.
    +    val expressions = ctx.expression.asScala.map { eCtx =>
    +      val e = expression(eCtx)
    +      assert(e.foldable, "All expressions in an inline table must be constants.", eCtx)
    +      e
    +    }
    +
    +    // Validate and evaluate the rows.
    +    val (structType, structConstructor) = expressions.head.dataType match {
    --- End diff --
    
    how do we make sure all expression are of same type?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-200562104
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53964/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57226156
  
    --- Diff: project/plugins.sbt ---
    @@ -23,3 +23,9 @@ libraryDependencies += "org.ow2.asm"  % "asm" % "5.0.3"
     libraryDependencies += "org.ow2.asm"  % "asm-commons" % "5.0.3"
     
     libraryDependencies += "org.antlr" % "antlr" % "3.5.2"
    +
    +
    +// TODO I am not sure we want such a dep.
    +resolvers += "simplytyped" at "http://simplytyped.github.io/repo/releases"
    --- End diff --
    
    We can do without the plugin (like we did for ANTLR3) but it'll take a little bit of effort. I'll probably end up doing this because I want ANTLR warnings to be treated like errors.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279041
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -0,0 +1,250 @@
    +/*
    + * 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.execution
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.misc.Interval
    +
    +import org.apache.spark.sql.SaveMode
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.parser.ng.{AbstractSqlParser, AstBuilder}
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.execution.command._
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * Concrete parser for Spark SQL statements.
    + */
    +object SparkSqlParser extends AbstractSqlParser{
    +  val astBuilder = new SparkSqlAstBuilder
    +}
    +
    +/**
    + * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier.
    + */
    +class SparkSqlAstBuilder extends AstBuilder {
    +  import AstBuilder._
    +  import org.apache.spark.sql.catalyst.parser.ParseUtils._
    +
    +  /**
    +   * Create a [[SetCommand]] logical plan.
    +   *
    +   * Note that we assume that everything after the SET keyword is assumed to be a part of the
    +   * key-value pair. The split between key and value is made by searching for the first `=`
    +   * character in the raw string.
    +   */
    +  override def visitSetConfiguration(ctx: SetConfigurationContext): LogicalPlan = withOrigin(ctx) {
    --- End diff --
    
    If we don't support a grammar in the parser. The parse rule is not implemented and returns a `null`; this is then captured by the 'ParseDriver` which will throw a ParseException.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202555073
  
    @hvanhovell Seems it breaks the build...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-200331100
  
    @hvanhovell tiny footnote: I double-checked that ANTLR4 is still BSD-licensed and it is. You might briefly update the entries in `LICENSE` to have one representative entry for ANTLR4. The artifact doesn't really matter (was just auto-generated once upon a time). It's not essential but good form.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202559495
  
    let me try to fix it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57077678
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1450 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ParseUtils
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import AstBuilder._
    +  import ParseUtils._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
    +    visit(ctx.dataType).asInstanceOf[DataType]
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          throw new ParseException("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(unescapeSQLString(pattern.getText)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }
    +
    +      // Check for duplicate names.
    +      ctes.groupBy(_._1).filter(_._2.size > 1).foreach {
    +        case (name, _) =>
    +          throw new ParseException(
    +            s"Name '$name' is used for multiple common table expressions", ctx)
    +      }
    +
    +      With(query, ctes.toMap)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryOrganization).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryOrganization).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).toSeq.flatMap {
    +      _.partitionVal.asScala.map {
    +        pVal =>
    +          val name = pVal.identifier.getText
    +          val value = Option(pVal.constant).map(c => expression(c).toString)
    +          name -> value
    +      }
    +    }.toMap
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan.
    +   */
    +  private def withQueryOrganization(
    --- End diff --
    
    It contains all the clauses that organize (order/distribute) the result of the query. I couldn't really think of something better.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57281611
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    --- End diff --
    
    It's not a big deal, we can keep it to make life easier :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57077787
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala ---
    @@ -161,6 +161,10 @@ package object dsl {
         def lower(e: Expression): Expression = Lower(e)
         def sqrt(e: Expression): Expression = Sqrt(e)
         def abs(e: Expression): Expression = Abs(e)
    +    def all(names: String*): Expression = names match {
    --- End diff --
    
    star is also fine.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193426445
  
    **[Test build #52572 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52572/consoleFull)** for PR 11557 at commit [`f95840d`](https://github.com/apache/spark/commit/f95840d00bcc2ba4324e9760907515e436b15edc).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202481765
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54318/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57147327
  
    --- Diff: project/plugins.sbt ---
    @@ -23,3 +23,9 @@ libraryDependencies += "org.ow2.asm"  % "asm" % "5.0.3"
     libraryDependencies += "org.ow2.asm"  % "asm-commons" % "5.0.3"
     
     libraryDependencies += "org.antlr" % "antlr" % "3.5.2"
    +
    +
    +// TODO I am not sure we want such a dep.
    +resolvers += "simplytyped" at "http://simplytyped.github.io/repo/releases"
    --- End diff --
    
    This is for the ANTLR4 SBT plugin. I am just curious if we want to depend on a github repo in our build.
    
    This will remain in the build even after we drop ANTLR3.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195808198
  
    **[Test build #53011 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53011/consoleFull)** for PR 11557 at commit [`5d4e13f`](https://github.com/apache/spark/commit/5d4e13f0a5161b6431936641649599e3f8f37134).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198919053
  
    **[Test build #53631 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53631/consoleFull)** for PR 11557 at commit [`4f1da75`](https://github.com/apache/spark/commit/4f1da75388424c6d4169af900655c769524bee02).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193182056
  
    cc @rxin @davies @andrewor14 @viirya 
    
    This will probably fail unit tests.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57147410
  
    --- Diff: sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/ng/SqlBase.g4 ---
    @@ -0,0 +1,742 @@
    +/*
    + * Licensed 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.
    + *
    + * This file is an adaptation of Presto's presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 grammar.
    + */
    +
    +grammar SqlBase;
    +
    +tokens {
    +    DELIMITER
    +}
    +
    +singleStatement
    +    : statement EOF
    +    ;
    +
    +singleExpression
    +    : namedExpression EOF
    +    ;
    +
    +singleTableIdentifier
    +    : tableIdentifier EOF
    +    ;
    +
    +singleDataType
    +    : dataType EOF
    +    ;
    +
    +statement
    +    : query                                                            #statementDefault
    +    | USE db=identifier                                                #use
    +    | createTable ('(' colTypeList ')')? tableProvider tableProperties #createTableUsing
    +    | createTable tableProvider tableProperties? AS? query             #createTableUsingAsSelect
    +    | DROP TABLE (IF EXISTS)? qualifiedName                            #dropTable
    +    | DELETE FROM qualifiedName (WHERE booleanExpression)?             #delete
    +    | ALTER TABLE from=qualifiedName RENAME TO to=qualifiedName        #renameTable
    +    | ALTER TABLE tableName=qualifiedName
    +        RENAME COLUMN from=identifier TO to=identifier                 #renameColumn
    +    | ALTER TABLE tableName=qualifiedName
    +        ADD COLUMN column=colType                                      #addColumn
    +    | CREATE (OR REPLACE)? VIEW qualifiedName AS query                 #createView
    +    | DROP VIEW (IF EXISTS)? qualifiedName                             #dropView
    +    | CALL qualifiedName '(' (callArgument (',' callArgument)*)? ')'   #call
    +    | EXPLAIN explainOption* statement                                 #explain
    +    | SHOW TABLES ((FROM | IN) db=identifier)?
    +        (LIKE (qualifiedName | pattern=STRING))?                       #showTables
    +    | SHOW SCHEMAS ((FROM | IN) identifier)?                           #showSchemas
    +    | SHOW CATALOGS                                                    #showCatalogs
    +    | SHOW COLUMNS (FROM | IN) qualifiedName                           #showColumns
    +    | SHOW FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))?         #showFunctions
    +    | (DESC | DESCRIBE) FUNCTION EXTENDED? qualifiedName               #describeFunction
    +    | (DESC | DESCRIBE) option=(EXTENDED | FORMATTED)?
    +        tableIdentifier partitionSpec? describeColName?                #describeTable
    +    | SHOW SESSION                                                     #showSession
    +    | SET SESSION qualifiedName EQ expression                          #setSession
    +    | RESET SESSION qualifiedName                                      #resetSession
    +    | START TRANSACTION (transactionMode (',' transactionMode)*)?      #startTransaction
    +    | COMMIT WORK?                                                     #commit
    +    | ROLLBACK WORK?                                                   #rollback
    +    | SHOW PARTITIONS (FROM | IN) qualifiedName
    +        (WHERE booleanExpression)?
    +        (ORDER BY sortItem (',' sortItem)*)?
    +        (LIMIT limit=(INTEGER_VALUE | ALL))?                           #showPartitions
    +    | REFRESH TABLE tableIdentifier                                    #refreshTable
    +    | CACHE LAZY? TABLE identifier (AS? query)?                        #cacheTable
    +    | UNCACHE TABLE identifier                                         #uncacheTable
    +    | CLEAR CACHE                                                      #clearCache
    +    | SET .*?                                                          #setConfiguration
    +    ;
    +
    +createTable
    +    : CREATE TEMPORARY? TABLE (IF NOT EXISTS)? tableIdentifier
    +    ;
    +
    +query
    +    : ctes? queryNoWith
    +    ;
    +
    +insertInto
    +    : INSERT OVERWRITE TABLE tableIdentifier partitionSpec? (IF NOT EXISTS)?
    +    | INSERT INTO TABLE? tableIdentifier partitionSpec?
    +    ;
    +
    +partitionSpec
    +    : PARTITION '(' partitionVal (',' partitionVal)* ')'
    +    ;
    +
    +partitionVal
    +    : identifier (EQ constant)?
    +    ;
    +
    +describeColName
    +    : identifier ('.' (identifier | STRING))*
    +    ;
    +
    +ctes
    +    : WITH namedQuery (',' namedQuery)*
    +    ;
    +
    +namedQuery
    +    : name=identifier AS? '(' queryNoWith ')'
    +    ;
    +
    +tableProvider
    +    : USING qualifiedName
    +    ;
    +
    +tableProperties
    +    :(OPTIONS | WITH) '(' tableProperty (',' tableProperty)* ')'
    +    ;
    +
    +tableProperty
    +    : key=tablePropertyKey (EQ? value=STRING)?
    +    ;
    +
    +tablePropertyKey
    +    : looseIdentifier ('.' looseIdentifier)*
    +    | STRING
    +    ;
    +
    +queryNoWith
    +    : insertInto? queryTerm queryOrganization                                              #singleInsertQuery
    +    | fromClause multiInsertQueryBody+                                                     #multiInsertQuery
    +    ;
    +
    +queryOrganization
    +    : (ORDER BY order+=sortItem (',' order+=sortItem)*)?
    +      (CLUSTER BY clusterBy+=expression (',' clusterBy+=expression)*)?
    +      (DISTRIBUTE BY distributeBy+=expression (',' distributeBy+=expression)*)?
    +      (SORT BY sort+=sortItem (',' sort+=sortItem)*)?
    +      windows?
    +      (LIMIT limit=expression)?
    +    ;
    +
    +multiInsertQueryBody
    +    : insertInto?
    +      querySpecification
    +      queryOrganization
    +    ;
    +
    +queryTerm
    +    : queryPrimary                                                                         #queryTermDefault
    +    | left=queryTerm operator=(INTERSECT | UNION | EXCEPT) setQuantifier? right=queryTerm  #setOperation
    +    ;
    +
    +queryPrimary
    +    : querySpecification                                                    #queryPrimaryDefault
    +    | TABLE tableIdentifier                                                 #table
    +    | inlineTable                                                           #inlineTableDefault1
    +    | '(' queryNoWith  ')'                                                  #subquery
    +    ;
    +
    +sortItem
    +    : expression ordering=(ASC | DESC)?
    +    ;
    +
    +querySpecification
    +    : (((SELECT kind=TRANSFORM | kind=MAP | kind=REDUCE)) '(' namedExpression (',' namedExpression)* ')'
    +       inRowFormat=rowFormat?
    +       USING script=STRING
    +       (AS (columnAliasList | colTypeList | ('(' (columnAliasList | colTypeList) ')')))?
    +       outRowFormat=rowFormat?
    +       (RECORDREADER outRecordReader=STRING)?
    +       fromClause?
    +       (WHERE where=booleanExpression)?)
    +    | (kind=SELECT setQuantifier? namedExpression (',' namedExpression)*
    +       fromClause?
    +       lateralView*
    +       (WHERE where=booleanExpression)?
    +       aggregation?
    +       (HAVING having=booleanExpression)?
    +       windows?)
    +    ;
    +
    +fromClause
    +    : FROM relation (',' relation)* lateralView*
    +    ;
    +
    +aggregation
    +    : GROUP BY groupingExpressions+=expression (',' groupingExpressions+=expression)* (
    +      WITH kind=ROLLUP
    +    | WITH kind=CUBE
    +    | kind=GROUPING SETS '(' groupingSet (',' groupingSet)* ')')?
    +    ;
    +
    +groupingSet
    +    : '(' (expression (',' expression)*)? ')'
    +    | expression
    +    ;
    +
    +lateralView
    +    : LATERAL VIEW (OUTER)? qualifiedName '(' (expression (',' expression)*)? ')' tblName=identifier (AS? colName+=identifier (',' colName+=identifier)*)
    +    ;
    +
    +setQuantifier
    +    : DISTINCT
    +    | ALL
    +    ;
    +
    +relation
    +    : left=relation
    +      ( CROSS JOIN right=sampledRelation
    +      | joinType JOIN rightRelation=relation joinCriteria?
    +      | NATURAL joinType JOIN right=sampledRelation
    +      )                                           #joinRelation
    +    | sampledRelation                             #relationDefault
    +    ;
    +
    +joinType
    +    : INNER?
    +    | LEFT OUTER?
    +    | LEFT SEMI
    +    | RIGHT OUTER?
    +    | FULL OUTER?
    +    ;
    +
    +joinCriteria
    +    : ON booleanExpression
    +    | USING '(' identifier (',' identifier)* ')'
    +    ;
    +
    +sampledRelation
    +    : relationPrimary (
    +        TABLESAMPLE '('
    +         ( (percentage=(INTEGER_VALUE | DECIMAL_VALUE) sampleType=PERCENTLIT)
    +         | (expression sampleType=ROWS)
    +         | (sampleType=BUCKET numerator=INTEGER_VALUE OUT OF denominator=INTEGER_VALUE (ON identifier)?))
    +         ')'
    +      )?
    +    ;
    +
    +columnAliases
    +    : '(' columnAliasList ')'
    +    ;
    +
    +columnAliasList
    +    : identifier (',' identifier)*
    +    ;
    +
    +relationPrimary
    +    : tableIdentifier (AS? identifier)?                             #tableName
    +    | '(' queryNoWith ')' (AS? identifier)?                         #aliasedQuery
    +    | '(' relation ')'  (AS? identifier)?                           #aliasedRelation
    +    | inlineTable                                                   #inlineTableDefault2
    +    ;
    +
    +inlineTable
    +    : VALUES expression (',' expression)*  (AS? identifier columnAliases?)?
    +    ;
    +
    +rowFormat
    +    : rowFormatSerde
    +    | rowFormatDelimited
    +    ;
    +
    +rowFormatSerde
    +    : ROW FORMAT SERDE name=STRING (WITH SERDEPROPERTIES props=tableProperties)?
    +    ;
    +
    +rowFormatDelimited
    +    : ROW FORMAT DELIMITED
    +      (FIELDS TERMINATED BY fieldsTerminatedBy=STRING)?
    +      (COLLECTION ITEMS TERMINATED BY collectionItemsTerminatedBy=STRING)?
    +      (MAP KEYS TERMINATED BY keysTerminatedBy=STRING)?
    +      (ESCAPED BY escapedBy=STRING)?
    +      (LINES SEPARATED BY linesSeparatedBy=STRING)?
    +    ;
    +
    +tableIdentifier
    +    : (db=identifier '.')? table=identifier
    +    ;
    +
    +namedExpression
    +    : expression (AS? (identifier | columnAliases))?
    +    ;
    +
    +expression
    +    : booleanExpression
    +    ;
    +
    +booleanExpression
    +    : predicated                                                   #booleanDefault
    +    | NOT booleanExpression                                        #logicalNot
    +    | left=booleanExpression operator=AND right=booleanExpression  #logicalBinary
    +    | left=booleanExpression operator=OR right=booleanExpression   #logicalBinary
    +    | EXISTS '(' query ')'                                         #exists
    +    ;
    +
    +// workaround for:
    +//  https://github.com/antlr/antlr4/issues/780
    +//  https://github.com/antlr/antlr4/issues/781
    +predicated
    +    : valueExpression predicate[$valueExpression.ctx]?
    +    ;
    +
    +predicate[ParserRuleContext value]
    +    : comparisonOperator right=valueExpression                            #comparison
    +    | NOT? BETWEEN lower=valueExpression AND upper=valueExpression        #between
    +    | NOT? IN '(' expression (',' expression)* ')'                        #inList
    +    | NOT? IN '(' query ')'                                               #inSubquery
    +    | NOT? like=(RLIKE | LIKE) pattern=valueExpression                    #like
    +    | IS NOT? NULL                                                        #nullPredicate
    +    ;
    +
    +valueExpression
    +    : primaryExpression                                                                      #valueExpressionDefault
    +    | operator=(MINUS | PLUS | TILDE) valueExpression                                        #arithmeticUnary
    +    | left=valueExpression operator=(ASTERISK | SLASH | PERCENT | DIV) right=valueExpression #arithmeticBinary
    +    | left=valueExpression operator=(PLUS | MINUS) right=valueExpression                     #arithmeticBinary
    +    | left=valueExpression operator=AMPERSAND right=valueExpression                          #arithmeticBinary
    +    | left=valueExpression operator=HAT right=valueExpression                                #arithmeticBinary
    +    | left=valueExpression operator=PIPE right=valueExpression                               #arithmeticBinary
    +    ;
    +
    +primaryExpression
    +    : constant                                                                                 #constantDefault
    +    | ASTERISK                                                                                 #star
    +    | qualifiedName '.' ASTERISK                                                               #star
    +    | '(' expression (',' expression)+ ')'                                                     #rowConstructor
    +    | qualifiedName '(' (ASTERISK) ')' (OVER windowSpec)?                                      #functionCall
    +    | qualifiedName '(' (setQuantifier? expression (',' expression)*)? ')' (OVER windowSpec)?  #functionCall
    +    | '(' query ')'                                                                            #subqueryExpression
    +    | CASE valueExpression whenClause+ (ELSE elseExpression=expression)? END                   #simpleCase
    +    | CASE whenClause+ (ELSE elseExpression=expression)? END                                   #searchedCase
    +    | CAST '(' expression AS dataType ')'                                                      #cast
    +    | value=primaryExpression '[' index=valueExpression ']'                                    #subscript
    +    | identifier                                                                               #columnReference
    +    | base=primaryExpression '.' fieldName=identifier                                          #dereference
    +    | '(' expression ')'                                                                       #parenthesizedExpression
    +    ;
    +
    +constant
    +    : NULL                                                                                     #nullLiteral
    +    | interval                                                                                 #intervalLiteral
    +    | identifier STRING                                                                        #typeConstructor
    +    | number                                                                                   #numericLiteral
    +    | booleanValue                                                                             #booleanLiteral
    +    | STRING+                                                                                  #stringLiteral
    +    ;
    +
    +comparisonOperator
    +    : EQ | NEQ | NEQJ | LT | LTE | GT | GTE | NSEQ
    +    ;
    +
    +booleanValue
    +    : TRUE | FALSE
    +    ;
    +
    +interval
    +    : INTERVAL intervalField*
    +    ;
    +
    +intervalField
    +    : value=intervalValue unit=identifier (TO to=identifier)?
    +    ;
    +
    +intervalValue
    +    : (PLUS | MINUS)? (INTEGER_VALUE | DECIMAL_VALUE)
    +    | STRING
    +    ;
    +
    +dataType
    +    : complex=ARRAY '<' dataType '>'                            #complexDataType
    +    | complex=MAP '<' dataType ',' dataType '>'                 #complexDataType
    +    | complex=STRUCT ('<' colTypeList? '>' | NEQ)              #complexDataType
    +    | identifier ('(' INTEGER_VALUE (',' INTEGER_VALUE)* ')')?  #primitiveDataType
    +    ;
    +
    +colTypeList
    +    : colType (',' colType)*
    +    ;
    +
    +colType
    +    : identifier ':'? dataType (COMMENT STRING)?
    +    ;
    +
    +whenClause
    +    : WHEN condition=expression THEN result=expression
    +    ;
    +
    +windows
    +    : WINDOW namedWindow (',' namedWindow)*
    +    ;
    +
    +namedWindow
    +    : identifier AS windowSpec
    +    ;
    +
    +windowSpec
    +    : name=identifier  #windowRef
    +    | '('
    +      (PARTITION BY partition+=expression (',' partition+=expression)*)?
    +      (ORDER BY sortItem (',' sortItem)* windowFrame?)?
    +      ')'              #windowDef
    +    ;
    +
    +windowFrame
    +    : frameType=RANGE start=frameBound
    +    | frameType=ROWS start=frameBound
    +    | frameType=RANGE BETWEEN start=frameBound AND end=frameBound
    +    | frameType=ROWS BETWEEN start=frameBound AND end=frameBound
    +    ;
    +
    +frameBound
    +    : UNBOUNDED boundType=(PRECEDING | FOLLOWING)
    +    | boundType=CURRENT ROW
    +    | expression boundType=(PRECEDING | FOLLOWING)
    +    ;
    +
    +
    +explainOption
    +    : LOGICAL | FORMATTED | EXTENDED
    +    ;
    +
    +transactionMode
    +    : ISOLATION LEVEL levelOfIsolation    #isolationLevel
    +    | READ accessMode=(ONLY | WRITE)      #transactionAccessMode
    +    ;
    +
    +levelOfIsolation
    +    : READ UNCOMMITTED                    #readUncommitted
    +    | READ COMMITTED                      #readCommitted
    +    | REPEATABLE READ                     #repeatableRead
    +    | SERIALIZABLE                        #serializable
    +    ;
    +
    +callArgument
    +    : expression                    #positionalArgument
    +    | identifier '=>' expression    #namedArgument
    +    ;
    +
    +qualifiedName
    +    : identifier ('.' identifier)*
    +    ;
    +
    +// Identifier that also allows the use of a number of SQL keywords (mainly for backwards compatibility).
    +looseIdentifier
    +    : identifier
    +    | FROM
    +    | TO
    +    | TABLE
    +    | WITH
    +    ;
    +
    +identifier
    +    : IDENTIFIER             #unquotedIdentifier
    +    | quotedIdentifier       #quotedIdentifierAlternative
    +    | nonReserved            #unquotedIdentifier
    +    ;
    +
    +quotedIdentifier
    +    : BACKQUOTED_IDENTIFIER
    +    ;
    +
    +number
    +    : DECIMAL_VALUE            #decimalLiteral
    +    | SCIENTIFIC_DECIMAL_VALUE #scientificDecimalLiteral
    +    | INTEGER_VALUE            #integerLiteral
    +    | BIGINT_LITERAL           #bigIntLiteral
    +    | SMALLINT_LITERAL         #smallIntLiteral
    +    | TINYINT_LITERAL          #tinyIntLiteral
    +    | DOUBLE_LITERAL           #doubleLiteral
    +    ;
    +
    +nonReserved
    +    : SHOW | TABLES | COLUMNS | COLUMN | PARTITIONS | FUNCTIONS | SCHEMAS | CATALOGS | SESSION
    +    | ADD
    +    | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | MAP | ARRAY | STRUCT
    +    | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER
    +    | DELIMITED | FIELDS | TERMINATED | COLLECTION | ITEMS | KEYS | ESCAPED | LINES | SEPARATED
    +    | EXTENDED | REFRESH | CLEAR | CACHE | UNCACHE | LAZY | TEMPORARY | OPTIONS
    +    | GROUPING | CUBE | ROLLUP
    +    | EXPLAIN | FORMAT | LOGICAL | FORMATTED
    +    | TABLESAMPLE | USE | TO | BUCKET | PERCENTLIT | OUT | OF
    +    | SET | RESET
    +    | VIEW | REPLACE
    +    | IF
    +    | NO | DATA
    +    | START | TRANSACTION | COMMIT | ROLLBACK | WORK | ISOLATION | LEVEL
    +    | SERIALIZABLE | REPEATABLE | COMMITTED | UNCOMMITTED | READ | WRITE | ONLY
    +    | CALL
    +    | SORT | CLUSTER | DISTRIBUTE
    +    ;
    +
    +SELECT: 'SELECT';
    --- End diff --
    
    should we move these keyword/lexer rules into a separated file?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55185915
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1128 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.sql.AnalysisException
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ParseUtils
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import AstBuilder._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          notSupported("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(unquote(pattern.getText)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }.toMap
    +      With(query, ctes)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryOrganization).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryOrganization).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).toSeq.flatMap {
    +      _.partitionVal.asScala.map {
    +        pVal => (pVal.identifier.getText, Option(pVal.constant).map(c => unquote(c.getText)))
    +      }
    +    }.toMap
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan.
    +   */
    +  private def withQueryOrganization(
    +      ctx: QueryOrganizationContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
    +    val withOrder = if (
    +      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // ORDER BY ...
    +      Sort(order.asScala.map(visitSortItem), global = true, query)
    +    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ...
    +      Sort(sort.asScala.map(visitSortItem), global = false, query)
    +    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // DISTRIBUTE BY ...
    +      RepartitionByExpression(expressionList(distributeBy), query)
    +    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ... DISTRIBUTE BY ...
    +      Sort(
    +        sort.asScala.map(visitSortItem),
    +        global = false,
    +        RepartitionByExpression(expressionList(distributeBy), query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
    +      // CLUSTER BY ...
    +      val expressions = expressionList(clusterBy)
    +      Sort(
    +        expressions.map(SortOrder(_, Ascending)),
    +        global = false,
    +        RepartitionByExpression(expressions, query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // [EMPTY]
    +      query
    +    } else {
    +      notSupported("Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
    +    }
    +
    +    // LIMIT
    +    val withLimit = withOrder.optional(limit) {
    +      Limit(typedVisit(limit), withOrder)
    +    }
    +
    +    // WINDOWS
    +    withLimit.optionalMap(windows)(withWindows)
    +  }
    +
    +  /**
    +   * Create a logical plan using a query specification.
    +   */
    +  override def visitQuerySpecification(
    +      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
    +    val from = OneRowRelation.optional(ctx.fromClause) {
    +      visitFromClause(ctx.fromClause)
    +    }
    +    withQuerySpecification(ctx, from)
    +  }
    +
    +  /**
    +   * Add a query specification to a logical plan. The query specification is the core of the logical
    +   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
    +   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
    +   *
    +   * Note that query hints are ignored (both by the parser and the builder).
    +   */
    +  private def withQuerySpecification(
    +      ctx: QuerySpecificationContext,
    +      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // WHERE
    +    val withFilter = relation.optional(where) {
    +      Filter(expression(where), relation)
    +    }
    +
    +    // Expressions.
    +    val expressions = namedExpression.asScala.map(visit).map {
    +      case e: Expression => UnresolvedAlias(e)
    +    }
    +
    +    // Create either a transform or a regular query.
    +    kind.getType match {
    +      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
    +        // Transform
    +
    +        // Create the attributes.
    +        val attributes = if (colTypeList != null) {
    +          // Typed return columns.
    +          visitColTypeList(colTypeList).toAttributes
    +        } else if (columnAliasList != null) {
    +          // Untyped return columns.
    +          visitColumnAliasList(columnAliasList).map { name =>
    +            AttributeReference(name, StringType, nullable = true)()
    +          }
    +        } else {
    +          Seq.empty
    +        }
    +
    +        // Create the transform.
    +        ScriptTransformation(
    +          expressions,
    +          unquote(script.getText),
    +          attributes,
    +          withFilter,
    +          withScriptIOSchema(inRowFormat, outRowFormat, outRecordReader))
    +
    +      case SqlBaseParser.SELECT =>
    +        // Regular select
    +
    +        // Add lateral views.
    +        val withLateralView = ctx.lateralView.asScala.foldLeft(withFilter)(withGenerate)
    +
    +        // Add aggregation with having or a project.
    +        val withProject = if (aggregation != null) {
    +          withAggregation(aggregation, expressions, withLateralView).optionalMap(having) {
    +            case (h, p) => Filter(expression(h), p)
    +          }
    +        } else {
    +          Project(expressions, withLateralView)
    +        }
    +
    +        // Distinct
    +        val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) {
    +          Distinct(withProject)
    +        } else {
    +          withProject
    +        }
    +
    +        // Window
    +        withDistinct.optionalMap(windows)(withWindows)
    +    }
    +  }
    +
    +  /**
    +   * Create a (Hive based) [[ScriptInputOutputSchema]].
    +   */
    +  protected def withScriptIOSchema(
    +      inRowFormat: RowFormatContext,
    +      outRowFormat: RowFormatContext,
    +      outRecordReader: Token): ScriptInputOutputSchema = null
    +
    +  /**
    +   * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
    +   * separated) relations here, these get converted into a single plan by condition-less inner join.
    +   */
    +  override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
    +    ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None))
    +  }
    +
    +  /**
    +   * Connect two queries by a Set operator.
    +   *
    +   * Supported Set operators are:
    +   * - UNION [DISTINCT]
    +   * - UNION ALL
    +   * - EXCEPT [DISTINCT]
    +   * - INTERSECT [DISTINCT]
    +   */
    +  override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
    +    val left = plan(ctx.left)
    +    val right = plan(ctx.right)
    +    val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.UNION if all =>
    +        Union(left, right)
    +      case SqlBaseParser.UNION =>
    +        Distinct(Union(left, right))
    +      case SqlBaseParser.INTERSECT if all =>
    +        notSupported("INTERSECT ALL is not supported.", ctx)
    +      case SqlBaseParser.INTERSECT =>
    +        Intersect(left, right)
    +      case SqlBaseParser.EXCEPT if all =>
    +        notSupported("EXCEPT ALL is not supported.", ctx)
    +      case SqlBaseParser.EXCEPT =>
    +        Except(left, right)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[WithWindowDefinition]] operator to a logical plan.
    +   */
    +  private def withWindows(
    +      ctx: WindowsContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Collect all window specifications defined in the WINDOW clause.
    +    val baseWindowMap = ctx.namedWindow.asScala.map {
    +      wCtx =>
    +        (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
    +    }.toMap
    +
    +    // Handle cases like
    +    // window w1 as (partition by p_mfgr order by p_name
    +    //               range between 2 preceding and 2 following),
    +    //        w2 as w1
    +    val windowMap = baseWindowMap.mapValues {
    +      case WindowSpecReference(name) => baseWindowMap(name).asInstanceOf[WindowSpecDefinition]
    +      case spec: WindowSpecDefinition => spec
    +    }
    +    WithWindowDefinition(windowMap, query)
    +  }
    +
    +  /**
    +   * Add an [[Aggregate]] to a logical plan.
    +   */
    +  private def withAggregation(
    +      ctx: AggregationContext,
    +      selectExpressions: Seq[NamedExpression],
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    val groupByExpressions = expressionList(groupingExpressions)
    +
    +    if (GROUPING != null) {
    +      // GROUP BY .... GROUPING SETS (...)
    +      // TODO use new expression set here?
    +      val expressionMap = groupByExpressions.zipWithIndex.toMap
    +      val numExpressions = expressionMap.size
    +      val mask = (1 << numExpressions) - 1
    +      val masks = ctx.groupingSet.asScala.map {
    +        _.expression.asScala.foldLeft(mask) {
    +          case (bitmap, eCtx) =>
    +            // Find the index of the expression.
    +            val e = typedVisit[Expression](eCtx)
    +            val index = expressionMap.find(_._1.semanticEquals(e)).map(_._2).getOrElse(
    +              throw new AnalysisException(
    +                s"${e.treeString} doesn't show up in the GROUP BY list"))
    +            // 0 means that the column at the given index is a grouping column, 1 means it is not,
    +            // so we unset the bit in bitmap.
    +            bitmap & ~(1 << (numExpressions - 1 - index))
    +        }
    +      }
    +      GroupingSets(masks, groupByExpressions, query, selectExpressions)
    +    } else {
    +      // GROUP BY .... (WITH CUBE | WITH ROLLUP)?
    +      val mappedGroupByExpressions = if (CUBE != null) {
    +        Seq(Cube(groupByExpressions))
    +      } else if (ROLLUP != null) {
    +        Seq(Rollup(groupByExpressions))
    +      } else {
    +        groupByExpressions
    +      }
    +      Aggregate(mappedGroupByExpressions, selectExpressions, query)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[Generate]] (Lateral View) to a logical plan.
    +   */
    +  private def withGenerate(
    +      query: LogicalPlan,
    +      ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
    +    val expressions = expressionList(ctx.expression)
    +
    +    // Create the generator.
    +    // TODO Add support for other generators.
    +    val generator = ctx.qualifiedName.getText.toLowerCase match {
    +      case "explode" if expressions.size == 1 =>
    +        Explode(expressions.head)
    +      case "json_tuple" =>
    +        JsonTuple(expressions)
    +      case other =>
    +        notSupported(s"Generator function '$other' is not supported", ctx)
    +    }
    +
    +    Generate(
    +      generator,
    +      join = true,
    +      outer = ctx.OUTER != null,
    +      Some(ctx.tblName.getText.toLowerCase),
    +      ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply),
    +      query)
    +  }
    +
    +  /**
    +   * Create a join between two logical plans.
    +   */
    +  override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) {
    +    val baseJoinType = ctx.joinType match {
    +      case jt if jt.FULL != null => FullOuter
    +      case jt if jt.SEMI != null => LeftSemi
    +      case jt if jt.LEFT != null => LeftOuter
    +      case jt if jt.RIGHT != null => RightOuter
    +      case _ => Inner
    +    }
    +    val joinType = if (ctx.NATURAL != null) {
    +      NaturalJoin(baseJoinType)
    +    } else {
    +      baseJoinType
    +    }
    +
    +    val left = plan(ctx.left)
    +    val right = if (ctx.right != null) {
    +      plan(ctx.right)
    +    } else {
    +      plan(ctx.rightRelation)
    +    }
    +    assert(left != null, "Left side should not be null", ctx)
    +    assert(right != null, "Right side should not be null", ctx)
    +    Join(left, right, joinType, Option(ctx.booleanExpression).map(expression))
    +  }
    +
    +  /**
    +   * Create a sampled relation. This returns a [[Sample]] operator when sampling is requested.
    +   *
    +   * This currently supports the following sampling methods:
    +   * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
    +   * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
    +   * are defined as a number between 0 and 100.
    +   * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
    +   */
    +  override def visitSampledRelation(ctx: SampledRelationContext): LogicalPlan = withOrigin(ctx) {
    +    val relation = plan(ctx.relationPrimary)
    +
    +    // Create a sampled plan if we need one.
    +    def sample(fraction: Double): Sample = {
    +      Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation)(true)
    +    }
    +
    +    // Sample the relation if we have to.
    +    relation.optional(ctx.sampleType) {
    +      ctx.sampleType.getType match {
    +        case SqlBaseParser.ROWS =>
    +          Limit(expression(ctx.expression), relation)
    +
    +        case SqlBaseParser.PERCENTLIT =>
    +          val fraction = ctx.percentage.getText.toDouble
    +          // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
    +          // function takes X PERCENT as the input and the range of X is [0, 100], we need to
    +          // adjust the fraction.
    +          val eps = RandomSampler.roundingEpsilon
    +          require(fraction >= 0.0 - eps && fraction <= 100.0 + eps,
    +            s"Sampling fraction ($fraction) must be on interval [0, 100]")
    +          sample(fraction / 100.0d)
    +
    +        case SqlBaseParser.BUCKET if ctx.ON != null =>
    +          notSupported("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx)
    +
    +        case SqlBaseParser.BUCKET =>
    +          sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a sub-query.
    +   */
    +  override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryNoWith)
    +  }
    +
    +  /**
    +   * Create an un-aliased table reference. This is typically used for top-level table references,
    +   * for example:
    +   * {{{
    +   *   INSERT INTO db.tbl2
    +   *   TABLE db.tbl1
    +   * }}}
    +   */
    +  override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None)
    +  }
    +
    +  /**
    +   * Create an aliased table reference. This is typically used in FROM clauses.
    +   */
    +  override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(
    +      visitTableIdentifier(ctx.tableIdentifier),
    +      Option(ctx.identifier).map(_.getText))
    +  }
    +
    +  /**
    +   * Create an inline table (a virtual table in Hive parlance).
    +   */
    +  override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
    +    // Get the backing expressions.
    +    val expressions = ctx.expression.asScala.map(expression)
    +
    +    // Validate and evaluate the rows.
    +    val (structType, structConstructor) = expressions.head.dataType match {
    +      case st: StructType =>
    +        (st, (e: Expression) => e)
    +      case dt =>
    +        val st = CreateStruct(Seq(expressions.head)).dataType
    +        (st, (e: Expression) => CreateStruct(Seq(e)))
    +    }
    +    val rows = expressions.map {
    +      case expression =>
    +        assert(expression.foldable, "All expressions in an inline table must be constants.", ctx)
    +        val safe = Cast(structConstructor(expression), structType)
    +        safe.eval().asInstanceOf[InternalRow]
    +    }
    +
    +    // Construct attributes.
    +    val baseAttributes = structType.toAttributes
    +    val attributes = if (ctx.columnAliases != null) {
    +      val aliases = visitColumnAliases(ctx.columnAliases)
    +      assert(aliases.size == baseAttributes.size,
    +        "Number of aliases must match the number of fields in an inline table.", ctx)
    +      baseAttributes.zip(aliases).map(p => p._1.withName(p._2))
    +    } else {
    +      baseAttributes
    +    }
    +
    +    LocalRelation(attributes, rows)
    +  }
    +
    +  /**
    +   * Create an alias (SubqueryAlias) for a join relation. This is practically the same as
    +   * visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different
    +   * hooks.
    +   */
    +  override def visitAliasedRelation(ctx: AliasedRelationContext): LogicalPlan = withOrigin(ctx) {
    +    aliasPlan(ctx.identifier, plan(ctx.relation()))
    +  }
    +
    +  /**
    +   * Create an alias (SubqueryAlias) for a sub-query. This is practically the same as
    +   * visitAliasedRelation and visitNamedExpression, ANTLR4 however requires us to use 3 different
    +   * hooks.
    +   */
    +  override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) {
    +    aliasPlan(ctx.identifier, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create an alias (SubqueryAlias) for a LogicalPlan. The alias is allowed to be optional.
    +   */
    +  private def aliasPlan(alias: IdentifierContext, plan: LogicalPlan): LogicalPlan = {
    +    plan.optional(alias) {
    +      SubqueryAlias(alias.getText, plan)
    +    }
    +  }
    +
    +  /**
    +   * Create a Sequence of Strings for a parenthesis enclosed alias list.
    +   */
    +  override def visitColumnAliases(ctx: ColumnAliasesContext): Seq[String] = withOrigin(ctx) {
    +    visitColumnAliasList(ctx.columnAliasList)
    +  }
    +
    +  /**
    +   * Create a Sequence of Strings for an alias list.
    +   */
    +  override def visitColumnAliasList(ctx: ColumnAliasListContext): Seq[String] = withOrigin(ctx) {
    +    ctx.identifier.asScala.map(_.getText)
    +  }
    +
    +  /**
    +   * Create a [[TableIdentifier]] from a 'tableName' or 'databaseName'.'tableName' pattern.
    +   */
    +  override def visitTableIdentifier(
    +      ctx: TableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText))
    +  }
    +
    +  /* ********************************************************************************************
    +   * Expression parsing
    +   * ******************************************************************************************** */
    +  private def expression(tree: ParserRuleContext): Expression = typedVisit(tree)
    +
    +  private def expressionList(trees: java.util.List[ExpressionContext]): Seq[Expression] = {
    +    trees.asScala.map(expression)
    +  }
    +
    +  private def invertIfNotDefined(expression: Expression, not: TerminalNode): Expression = {
    +    if (not != null) {
    +      Not(expression)
    +    } else {
    +      expression
    +    }
    +  }
    +
    +  override def visitStar(ctx: StarContext): Expression = withOrigin(ctx) {
    +    UnresolvedStar(Option(ctx.qualifiedName()).map(_.identifier.asScala.map(_.getText)))
    +  }
    +
    +  override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) {
    +    val e = expression(ctx.expression)
    +    if (ctx.identifier != null) {
    +      Alias(e, ctx.identifier.getText)()
    +    } else if (ctx.columnAliases != null) {
    +      MultiAlias(e, visitColumnAliases(ctx.columnAliases))
    +    } else {
    +      e
    +    }
    +  }
    +
    +  override def visitLogicalBinary(ctx: LogicalBinaryContext): Expression = withOrigin(ctx) {
    +    val left = expression(ctx.left)
    +    val right = expression(ctx.right)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.AND =>
    +        And(left, right)
    +      case SqlBaseParser.OR =>
    +        Or(left, right)
    +    }
    --- End diff --
    
    We should flatten OR/AND's here, see: https://github.com/apache/spark/pull/11501


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57071107
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1450 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ParseUtils
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import AstBuilder._
    +  import ParseUtils._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
    +    visit(ctx.dataType).asInstanceOf[DataType]
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          throw new ParseException("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(unescapeSQLString(pattern.getText)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }
    +
    +      // Check for duplicate names.
    +      ctes.groupBy(_._1).filter(_._2.size > 1).foreach {
    +        case (name, _) =>
    +          throw new ParseException(
    +            s"Name '$name' is used for multiple common table expressions", ctx)
    +      }
    +
    +      With(query, ctes.toMap)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryOrganization).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryOrganization).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).toSeq.flatMap {
    +      _.partitionVal.asScala.map {
    +        pVal =>
    +          val name = pVal.identifier.getText
    +          val value = Option(pVal.constant).map(c => expression(c).toString)
    +          name -> value
    +      }
    +    }.toMap
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan.
    +   */
    +  private def withQueryOrganization(
    --- End diff --
    
    why is this called QueryOrganization?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55195499
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/functions.scala ---
    @@ -1953,7 +1954,6 @@ object functions extends LegacyFunctions {
        *
        * @param e column to compute SHA-2 on.
        * @param numBits one of 224, 256, 384, or 512.
    -   *
    --- End diff --
    
    Intellij always does this for some reason :S


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55195847
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala ---
    @@ -331,8 +331,8 @@ class JoinSuite extends QueryTest with SharedSQLContext {
       }
     
       test("full outer join") {
    -    upperCaseData.where('N <= 4).registerTempTable("left")
    -    upperCaseData.where('N >= 3).registerTempTable("right")
    +    upperCaseData.where('N <= 4).registerTempTable("`left`")
    --- End diff --
    
    `LEFT`/`RIGHT` are SQL keywords. The old parser is a bit more lenient.
    
    The question is if we should allow SQL keywords in table names? I'd prefer not support this. If we do, then we should move identifier parsing into a seperate parser.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193368143
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-200561508
  
    **[Test build #53964 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53964/consoleFull)** for PR 11557 at commit [`be6f5c8`](https://github.com/apache/spark/commit/be6f5c82956152ac33c78d484ee5fe8952e18632).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193520951
  
    **[Test build #52594 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52594/consoleFull)** for PR 11557 at commit [`340522e`](https://github.com/apache/spark/commit/340522eeb37e36bc50d41152b4ec274c8ced3095).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/11557


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198167846
  
    **[Test build #53488 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53488/consoleFull)** for PR 11557 at commit [`c5d0bcf`](https://github.com/apache/spark/commit/c5d0bcff0d41c1f302d842d6e111fa85a8baf74b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202555657
  
    https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-2.3/956/consoleFull
    
    ```
    [error] /home/jenkins/workspace/spark-master-compile-maven-hadoop-2.3/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala:31: object SqlBaseParser is not a member of package org.apache.spark.sql.catalyst.parser.ng
    [error] import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    [error]                                                ^
    [error] /home/jenkins/workspace/spark-master-compile-maven-hadoop-2.3/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala:42: not found: type SqlBaseBaseVisitor
    [error] class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    [error]                          ^
    [error] /home/jenkins/workspace/spark-master-compile-maven-hadoop-2.3/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala:46: type mismatch;
    [error]  found   : org.apache.spark.sql.catalyst.parser.ng.AstBuilder
    [error]  required: org.antlr.v4.runtime.tree.ParseTreeVisitor[_]
    [error]     ctx.accept(this).asInstanceOf[T]
    [error]                ^
    [error] /home/jenkins/workspace/spark-master-compile-maven-hadoop-2.3/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala:49: not found: type SingleStatementContext
    [error]   override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    [error]                                          ^
    [error] /home/jenkins/workspace/spark-master-compile-maven-hadoop-2.3/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala:53: not found: type SingleExpressionContext
    [error]   override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    [error]                                           ^
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193311272
  
    **[Test build #52567 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52567/consoleFull)** for PR 11557 at commit [`c105f83`](https://github.com/apache/spark/commit/c105f83c12078adf2c3e3388f00797c43ee87dd5).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57278951
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    --- End diff --
    
    This currently works in our parser. I first tied a `HAVING` to a `GROUP BY` clause, but then the following query failed: `SELECT COUNT(*), SUM(A) FROM tbl HAVING COUNT(*) > 1`
    
    I have also added this to show that some of the syntax is a bit weird.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57281018
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    --- End diff --
    
    Throwing an exception during analysis would actually be a regression. I am pretty sure that we support this in HiveQl for 1.6.
    
    I am also not sure if we should be that struct. HAVING only means (IMHO) that a filter is applied after a potential aggregation.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57148383
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/CatalystQlSuite.scala ---
    @@ -21,15 +21,18 @@ import org.apache.spark.sql.AnalysisException
     import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis._
     import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical._
     import org.apache.spark.unsafe.types.CalendarInterval
     
     class CatalystQlSuite extends PlanTest {
    -  val parser = new CatalystQl()
    +  val parser = CatalystSqlParser
    --- End diff --
    
    And these dsl simplifications LGTM, we can still keep them here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55195410
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala ---
    @@ -0,0 +1,250 @@
    +/*
    + * 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.execution
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.misc.Interval
    +
    +import org.apache.spark.sql.SaveMode
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
    +import org.apache.spark.sql.catalyst.parser.ng.{AbstractSqlParser, AstBuilder}
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
    +import org.apache.spark.sql.execution.command._
    +import org.apache.spark.sql.execution.datasources._
    +
    +/**
    + * Concrete parser for Spark SQL statements.
    + */
    +object SparkSqlParser extends AbstractSqlParser{
    +  val astBuilder = new SparkSqlAstBuilder
    +}
    +
    +/**
    + * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier.
    + */
    +class SparkSqlAstBuilder extends AstBuilder {
    +  import AstBuilder._
    +
    +  /**
    +   * Create a [[SetCommand]] logical plan.
    +   *
    +   * Note that we assume that everything after the SET keyword is assumed to be a part of the
    +   * key-value pair. The split between key and value is made by searching for the first `=`
    +   * character in the raw string.
    +   */
    +  override def visitSetConfiguration(ctx: SetConfigurationContext): LogicalPlan = withOrigin(ctx) {
    +    // Get the remaining text from the stream.
    +    val stream = ctx.getStop.getInputStream
    +    val interval = Interval.of(ctx.getStart.getStopIndex + 1, stream.size())
    +    val remainder = stream.getText(interval)
    +
    +    // Construct the command.
    +    val keyValueSeparatorIndex = remainder.indexOf('=')
    +    if (keyValueSeparatorIndex >= 0) {
    +      val key = remainder.substring(0, keyValueSeparatorIndex).trim
    +      val value = remainder.substring(keyValueSeparatorIndex + 1).trim
    +      SetCommand(Some(key -> Option(value)))
    +    } else if (remainder.nonEmpty) {
    +      SetCommand(Some(remainder.trim -> None))
    +    } else {
    +      SetCommand(None)
    +    }
    +  }
    +
    +  /**
    +   * Create a [[SetDatabaseCommand]] logical plan.
    +   */
    +  override def visitUse(ctx: UseContext): LogicalPlan = withOrigin(ctx) {
    +    SetDatabaseCommand(ctx.db.getText)
    +  }
    +
    +  /**
    +   * Create a [[ShowTablesCommand]] logical plan.
    +   */
    +  override def visitShowTables(ctx: ShowTablesContext): LogicalPlan = withOrigin(ctx) {
    +    if (ctx.LIKE != null) {
    +      logWarning("SHOW TABLES LIKE option is ignored.")
    +    }
    +    ShowTablesCommand(Option(ctx.db).map(_.getText))
    +  }
    +
    +  /**
    +   * Create a [[RefreshTable]] logical plan.
    +   */
    +  override def visitRefreshTable(ctx: RefreshTableContext): LogicalPlan = withOrigin(ctx) {
    +    RefreshTable(visitTableIdentifier(ctx.tableIdentifier))
    +  }
    +
    +  /**
    +   * Create a [[CacheTableCommand]] logical plan.
    +   */
    +  override def visitCacheTable(ctx: CacheTableContext): LogicalPlan = withOrigin(ctx) {
    +    val query = Option(ctx.query).map(plan)
    +    CacheTableCommand(ctx.identifier.getText, query, ctx.LAZY != null)
    +  }
    +
    +  /**
    +   * Create an [[UncacheTableCommand]] logical plan.
    +   */
    +  override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) {
    +    UncacheTableCommand(ctx.identifier.getText)
    +  }
    +
    +  /**
    +   * Create a [[ClearCacheCommand]] logical plan.
    +   */
    +  override def visitClearCache(ctx: ClearCacheContext): LogicalPlan = withOrigin(ctx) {
    +    ClearCacheCommand
    +  }
    +
    +  /**
    +   * Create an [[ExplainCommand]] logical plan.
    +   */
    +  override def visitExplain(ctx: ExplainContext): LogicalPlan = withOrigin(ctx) {
    +    val options = ctx.explainOption.asScala
    +    if (options.exists(_.FORMATTED != null)) {
    +      logWarning("EXPLAIN FORMATTED option is ignored.")
    +    }
    +    if (options.exists(_.LOGICAL != null)) {
    +      logWarning("EXPLAIN LOGICAL option is ignored.")
    +    }
    +
    +    // Create the explain comment.
    +    val statement = plan(ctx.statement)
    +    if (isExplainableStatement(statement)) {
    +      ExplainCommand(statement, extended = options.exists(_.EXTENDED != null))
    +    } else {
    +      ExplainCommand(OneRowRelation)
    +    }
    +  }
    +
    +  /**
    +   * Determine if a plan should be explained at all.
    +   */
    +  protected def isExplainableStatement(plan: LogicalPlan): Boolean = plan match {
    +    case _: datasources.DescribeCommand => false
    +    case _ => true
    +  }
    +
    +  /**
    +   * Create a [[DescribeCommand]] logical plan.
    +   */
    +  override def visitDescribeTable(ctx: DescribeTableContext): LogicalPlan = withOrigin(ctx) {
    +    // FORMATTED and columns are not supported. Return null and let the parser decide what to do
    +    // with this (create an exception or pass it on to a different system).
    +    if (ctx.describeColName != null || ctx.FORMATTED != null) {
    +      null
    +    } else {
    +      // Partitioning clause is ignored.
    +      if (ctx.partitionSpec != null) {
    +        logWarning("DESCRIBE PARTITIONING option is ignored.")
    +      }
    +      datasources.DescribeCommand(
    +        UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None),
    +        ctx.EXTENDED != null)
    +    }
    +  }
    +
    +  /**
    +   * Validate a create table statement and return the [[TableIdentifier]].
    +   */
    +  override def visitCreateTable(
    +      ctx: CreateTableContext): (TableIdentifier, Boolean, Boolean) = withOrigin(ctx) {
    +    val temporary = ctx.TEMPORARY != null
    +    val ifNotExists = ctx.EXISTS != null
    +    assert(!temporary || !ifNotExists,
    +      "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.",
    +      ctx)
    +    (visitTableIdentifier(ctx.tableIdentifier), temporary, ifNotExists)
    +  }
    +
    +  /**
    +   * Create a [[CreateTableUsing]] logical plan.
    +   */
    +  override def visitCreateTableUsing(ctx: CreateTableUsingContext): LogicalPlan = withOrigin(ctx) {
    +    val (table, temporary, ifNotExists) = visitCreateTable(ctx.createTable)
    +    val options = Option(ctx.tableProperties)
    +      .map(visitTableProperties)
    +      .getOrElse(Map.empty)
    +    CreateTableUsing(
    +      table,
    +      Option(ctx.colTypeList).map(visitColTypeList),
    +      ctx.tableProvider.qualifiedName.getText,
    +      temporary,
    +      options,
    +      ifNotExists,
    --- End diff --
    
    This is a bit funky. The name of the parameter is `allowExisting`, while we are passing in a flag based on the existance of an `IF NOT EXISTS` clause (which is the opposite IMHO).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57070615
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala ---
    @@ -161,6 +161,10 @@ package object dsl {
         def lower(e: Expression): Expression = Lower(e)
         def sqrt(e: Expression): Expression = Sqrt(e)
         def abs(e: Expression): Expression = Abs(e)
    +    def all(names: String*): Expression = names match {
    --- End diff --
    
    star maybe?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195862507
  
    **[Test build #53020 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53020/consoleFull)** for PR 11557 at commit [`303394f`](https://github.com/apache/spark/commit/303394f6c114eda0c085c871dd34b7ae70e9c81f).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55417347
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1128 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.sql.AnalysisException
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ParseUtils
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import AstBuilder._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          notSupported("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(unquote(pattern.getText)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }.toMap
    +      With(query, ctes)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryOrganization).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryOrganization).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).toSeq.flatMap {
    +      _.partitionVal.asScala.map {
    +        pVal => (pVal.identifier.getText, Option(pVal.constant).map(c => unquote(c.getText)))
    +      }
    +    }.toMap
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan.
    +   */
    +  private def withQueryOrganization(
    +      ctx: QueryOrganizationContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
    +    val withOrder = if (
    +      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // ORDER BY ...
    +      Sort(order.asScala.map(visitSortItem), global = true, query)
    +    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ...
    +      Sort(sort.asScala.map(visitSortItem), global = false, query)
    +    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // DISTRIBUTE BY ...
    +      RepartitionByExpression(expressionList(distributeBy), query)
    +    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ... DISTRIBUTE BY ...
    +      Sort(
    +        sort.asScala.map(visitSortItem),
    +        global = false,
    +        RepartitionByExpression(expressionList(distributeBy), query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
    +      // CLUSTER BY ...
    +      val expressions = expressionList(clusterBy)
    +      Sort(
    +        expressions.map(SortOrder(_, Ascending)),
    +        global = false,
    +        RepartitionByExpression(expressions, query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // [EMPTY]
    +      query
    +    } else {
    +      notSupported("Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
    +    }
    +
    +    // LIMIT
    +    val withLimit = withOrder.optional(limit) {
    +      Limit(typedVisit(limit), withOrder)
    +    }
    +
    +    // WINDOWS
    +    withLimit.optionalMap(windows)(withWindows)
    +  }
    +
    +  /**
    +   * Create a logical plan using a query specification.
    +   */
    +  override def visitQuerySpecification(
    +      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
    +    val from = OneRowRelation.optional(ctx.fromClause) {
    +      visitFromClause(ctx.fromClause)
    +    }
    +    withQuerySpecification(ctx, from)
    +  }
    +
    +  /**
    +   * Add a query specification to a logical plan. The query specification is the core of the logical
    +   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
    +   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
    +   *
    +   * Note that query hints are ignored (both by the parser and the builder).
    +   */
    +  private def withQuerySpecification(
    +      ctx: QuerySpecificationContext,
    +      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // WHERE
    +    val withFilter = relation.optional(where) {
    +      Filter(expression(where), relation)
    +    }
    +
    +    // Expressions.
    +    val expressions = namedExpression.asScala.map(visit).map {
    +      case e: Expression => UnresolvedAlias(e)
    +    }
    +
    +    // Create either a transform or a regular query.
    +    kind.getType match {
    +      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
    +        // Transform
    +
    +        // Create the attributes.
    +        val attributes = if (colTypeList != null) {
    +          // Typed return columns.
    +          visitColTypeList(colTypeList).toAttributes
    +        } else if (columnAliasList != null) {
    +          // Untyped return columns.
    +          visitColumnAliasList(columnAliasList).map { name =>
    +            AttributeReference(name, StringType, nullable = true)()
    +          }
    +        } else {
    +          Seq.empty
    +        }
    +
    +        // Create the transform.
    +        ScriptTransformation(
    +          expressions,
    +          unquote(script.getText),
    +          attributes,
    +          withFilter,
    +          withScriptIOSchema(inRowFormat, outRowFormat, outRecordReader))
    +
    +      case SqlBaseParser.SELECT =>
    +        // Regular select
    +
    +        // Add lateral views.
    +        val withLateralView = ctx.lateralView.asScala.foldLeft(withFilter)(withGenerate)
    +
    +        // Add aggregation with having or a project.
    +        val withProject = if (aggregation != null) {
    +          withAggregation(aggregation, expressions, withLateralView).optionalMap(having) {
    +            case (h, p) => Filter(expression(h), p)
    +          }
    +        } else {
    +          Project(expressions, withLateralView)
    +        }
    +
    +        // Distinct
    +        val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) {
    +          Distinct(withProject)
    +        } else {
    +          withProject
    +        }
    +
    +        // Window
    +        withDistinct.optionalMap(windows)(withWindows)
    +    }
    +  }
    +
    +  /**
    +   * Create a (Hive based) [[ScriptInputOutputSchema]].
    +   */
    +  protected def withScriptIOSchema(
    +      inRowFormat: RowFormatContext,
    +      outRowFormat: RowFormatContext,
    +      outRecordReader: Token): ScriptInputOutputSchema = null
    +
    +  /**
    +   * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
    +   * separated) relations here, these get converted into a single plan by condition-less inner join.
    +   */
    +  override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
    +    ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None))
    +  }
    +
    +  /**
    +   * Connect two queries by a Set operator.
    +   *
    +   * Supported Set operators are:
    +   * - UNION [DISTINCT]
    +   * - UNION ALL
    +   * - EXCEPT [DISTINCT]
    +   * - INTERSECT [DISTINCT]
    +   */
    +  override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
    +    val left = plan(ctx.left)
    +    val right = plan(ctx.right)
    +    val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.UNION if all =>
    +        Union(left, right)
    +      case SqlBaseParser.UNION =>
    +        Distinct(Union(left, right))
    +      case SqlBaseParser.INTERSECT if all =>
    +        notSupported("INTERSECT ALL is not supported.", ctx)
    +      case SqlBaseParser.INTERSECT =>
    +        Intersect(left, right)
    +      case SqlBaseParser.EXCEPT if all =>
    +        notSupported("EXCEPT ALL is not supported.", ctx)
    +      case SqlBaseParser.EXCEPT =>
    +        Except(left, right)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[WithWindowDefinition]] operator to a logical plan.
    +   */
    +  private def withWindows(
    +      ctx: WindowsContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Collect all window specifications defined in the WINDOW clause.
    +    val baseWindowMap = ctx.namedWindow.asScala.map {
    +      wCtx =>
    +        (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
    +    }.toMap
    +
    +    // Handle cases like
    +    // window w1 as (partition by p_mfgr order by p_name
    +    //               range between 2 preceding and 2 following),
    +    //        w2 as w1
    +    val windowMap = baseWindowMap.mapValues {
    +      case WindowSpecReference(name) => baseWindowMap(name).asInstanceOf[WindowSpecDefinition]
    +      case spec: WindowSpecDefinition => spec
    +    }
    +    WithWindowDefinition(windowMap, query)
    +  }
    +
    +  /**
    +   * Add an [[Aggregate]] to a logical plan.
    +   */
    +  private def withAggregation(
    +      ctx: AggregationContext,
    +      selectExpressions: Seq[NamedExpression],
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    val groupByExpressions = expressionList(groupingExpressions)
    +
    +    if (GROUPING != null) {
    +      // GROUP BY .... GROUPING SETS (...)
    +      // TODO use new expression set here?
    +      val expressionMap = groupByExpressions.zipWithIndex.toMap
    +      val numExpressions = expressionMap.size
    +      val mask = (1 << numExpressions) - 1
    +      val masks = ctx.groupingSet.asScala.map {
    +        _.expression.asScala.foldLeft(mask) {
    +          case (bitmap, eCtx) =>
    +            // Find the index of the expression.
    +            val e = typedVisit[Expression](eCtx)
    +            val index = expressionMap.find(_._1.semanticEquals(e)).map(_._2).getOrElse(
    +              throw new AnalysisException(
    +                s"${e.treeString} doesn't show up in the GROUP BY list"))
    +            // 0 means that the column at the given index is a grouping column, 1 means it is not,
    +            // so we unset the bit in bitmap.
    +            bitmap & ~(1 << (numExpressions - 1 - index))
    +        }
    +      }
    +      GroupingSets(masks, groupByExpressions, query, selectExpressions)
    +    } else {
    +      // GROUP BY .... (WITH CUBE | WITH ROLLUP)?
    +      val mappedGroupByExpressions = if (CUBE != null) {
    +        Seq(Cube(groupByExpressions))
    +      } else if (ROLLUP != null) {
    +        Seq(Rollup(groupByExpressions))
    +      } else {
    +        groupByExpressions
    +      }
    +      Aggregate(mappedGroupByExpressions, selectExpressions, query)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[Generate]] (Lateral View) to a logical plan.
    +   */
    +  private def withGenerate(
    +      query: LogicalPlan,
    +      ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
    +    val expressions = expressionList(ctx.expression)
    +
    +    // Create the generator.
    +    // TODO Add support for other generators.
    +    val generator = ctx.qualifiedName.getText.toLowerCase match {
    +      case "explode" if expressions.size == 1 =>
    +        Explode(expressions.head)
    +      case "json_tuple" =>
    +        JsonTuple(expressions)
    +      case other =>
    +        notSupported(s"Generator function '$other' is not supported", ctx)
    +    }
    +
    +    Generate(
    +      generator,
    +      join = true,
    +      outer = ctx.OUTER != null,
    +      Some(ctx.tblName.getText.toLowerCase),
    +      ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply),
    +      query)
    +  }
    +
    +  /**
    +   * Create a join between two logical plans.
    +   */
    +  override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) {
    +    val baseJoinType = ctx.joinType match {
    +      case jt if jt.FULL != null => FullOuter
    +      case jt if jt.SEMI != null => LeftSemi
    +      case jt if jt.LEFT != null => LeftOuter
    +      case jt if jt.RIGHT != null => RightOuter
    +      case _ => Inner
    +    }
    +    val joinType = if (ctx.NATURAL != null) {
    +      NaturalJoin(baseJoinType)
    +    } else {
    +      baseJoinType
    +    }
    +
    +    val left = plan(ctx.left)
    +    val right = if (ctx.right != null) {
    +      plan(ctx.right)
    +    } else {
    +      plan(ctx.rightRelation)
    +    }
    +    assert(left != null, "Left side should not be null", ctx)
    +    assert(right != null, "Right side should not be null", ctx)
    +    Join(left, right, joinType, Option(ctx.booleanExpression).map(expression))
    +  }
    +
    +  /**
    +   * Create a sampled relation. This returns a [[Sample]] operator when sampling is requested.
    +   *
    +   * This currently supports the following sampling methods:
    +   * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
    +   * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
    +   * are defined as a number between 0 and 100.
    +   * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
    +   */
    +  override def visitSampledRelation(ctx: SampledRelationContext): LogicalPlan = withOrigin(ctx) {
    +    val relation = plan(ctx.relationPrimary)
    +
    +    // Create a sampled plan if we need one.
    +    def sample(fraction: Double): Sample = {
    +      Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation)(true)
    +    }
    +
    +    // Sample the relation if we have to.
    +    relation.optional(ctx.sampleType) {
    +      ctx.sampleType.getType match {
    +        case SqlBaseParser.ROWS =>
    +          Limit(expression(ctx.expression), relation)
    +
    +        case SqlBaseParser.PERCENTLIT =>
    +          val fraction = ctx.percentage.getText.toDouble
    +          // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
    +          // function takes X PERCENT as the input and the range of X is [0, 100], we need to
    +          // adjust the fraction.
    +          val eps = RandomSampler.roundingEpsilon
    +          require(fraction >= 0.0 - eps && fraction <= 100.0 + eps,
    +            s"Sampling fraction ($fraction) must be on interval [0, 100]")
    +          sample(fraction / 100.0d)
    +
    +        case SqlBaseParser.BUCKET if ctx.ON != null =>
    +          notSupported("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx)
    +
    +        case SqlBaseParser.BUCKET =>
    +          sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a sub-query.
    +   */
    +  override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryNoWith)
    +  }
    +
    +  /**
    +   * Create an un-aliased table reference. This is typically used for top-level table references,
    +   * for example:
    +   * {{{
    +   *   INSERT INTO db.tbl2
    +   *   TABLE db.tbl1
    +   * }}}
    +   */
    +  override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None)
    +  }
    +
    +  /**
    +   * Create an aliased table reference. This is typically used in FROM clauses.
    +   */
    +  override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(
    +      visitTableIdentifier(ctx.tableIdentifier),
    +      Option(ctx.identifier).map(_.getText))
    +  }
    +
    +  /**
    +   * Create an inline table (a virtual table in Hive parlance).
    +   */
    +  override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
    +    // Get the backing expressions.
    +    val expressions = ctx.expression.asScala.map(expression)
    +
    +    // Validate and evaluate the rows.
    +    val (structType, structConstructor) = expressions.head.dataType match {
    +      case st: StructType =>
    +        (st, (e: Expression) => e)
    +      case dt =>
    +        val st = CreateStruct(Seq(expressions.head)).dataType
    +        (st, (e: Expression) => CreateStruct(Seq(e)))
    +    }
    +    val rows = expressions.map {
    +      case expression =>
    +        assert(expression.foldable, "All expressions in an inline table must be constants.", ctx)
    +        val safe = Cast(structConstructor(expression), structType)
    +        safe.eval().asInstanceOf[InternalRow]
    +    }
    +
    +    // Construct attributes.
    +    val baseAttributes = structType.toAttributes
    +    val attributes = if (ctx.columnAliases != null) {
    +      val aliases = visitColumnAliases(ctx.columnAliases)
    +      assert(aliases.size == baseAttributes.size,
    +        "Number of aliases must match the number of fields in an inline table.", ctx)
    +      baseAttributes.zip(aliases).map(p => p._1.withName(p._2))
    +    } else {
    +      baseAttributes
    +    }
    +
    +    LocalRelation(attributes, rows)
    +  }
    +
    +  /**
    +   * Create an alias (SubqueryAlias) for a join relation. This is practically the same as
    +   * visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different
    +   * hooks.
    +   */
    +  override def visitAliasedRelation(ctx: AliasedRelationContext): LogicalPlan = withOrigin(ctx) {
    +    aliasPlan(ctx.identifier, plan(ctx.relation()))
    +  }
    +
    +  /**
    +   * Create an alias (SubqueryAlias) for a sub-query. This is practically the same as
    +   * visitAliasedRelation and visitNamedExpression, ANTLR4 however requires us to use 3 different
    +   * hooks.
    +   */
    +  override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) {
    +    aliasPlan(ctx.identifier, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create an alias (SubqueryAlias) for a LogicalPlan. The alias is allowed to be optional.
    +   */
    +  private def aliasPlan(alias: IdentifierContext, plan: LogicalPlan): LogicalPlan = {
    +    plan.optional(alias) {
    +      SubqueryAlias(alias.getText, plan)
    +    }
    +  }
    +
    +  /**
    +   * Create a Sequence of Strings for a parenthesis enclosed alias list.
    +   */
    +  override def visitColumnAliases(ctx: ColumnAliasesContext): Seq[String] = withOrigin(ctx) {
    +    visitColumnAliasList(ctx.columnAliasList)
    +  }
    +
    +  /**
    +   * Create a Sequence of Strings for an alias list.
    +   */
    +  override def visitColumnAliasList(ctx: ColumnAliasListContext): Seq[String] = withOrigin(ctx) {
    +    ctx.identifier.asScala.map(_.getText)
    +  }
    +
    +  /**
    +   * Create a [[TableIdentifier]] from a 'tableName' or 'databaseName'.'tableName' pattern.
    +   */
    +  override def visitTableIdentifier(
    +      ctx: TableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText))
    +  }
    +
    +  /* ********************************************************************************************
    +   * Expression parsing
    +   * ******************************************************************************************** */
    +  private def expression(tree: ParserRuleContext): Expression = typedVisit(tree)
    +
    +  private def expressionList(trees: java.util.List[ExpressionContext]): Seq[Expression] = {
    +    trees.asScala.map(expression)
    +  }
    +
    +  private def invertIfNotDefined(expression: Expression, not: TerminalNode): Expression = {
    +    if (not != null) {
    +      Not(expression)
    +    } else {
    +      expression
    +    }
    +  }
    +
    +  override def visitStar(ctx: StarContext): Expression = withOrigin(ctx) {
    +    UnresolvedStar(Option(ctx.qualifiedName()).map(_.identifier.asScala.map(_.getText)))
    +  }
    +
    +  override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) {
    +    val e = expression(ctx.expression)
    +    if (ctx.identifier != null) {
    +      Alias(e, ctx.identifier.getText)()
    +    } else if (ctx.columnAliases != null) {
    +      MultiAlias(e, visitColumnAliases(ctx.columnAliases))
    +    } else {
    +      e
    +    }
    +  }
    +
    +  override def visitLogicalBinary(ctx: LogicalBinaryContext): Expression = withOrigin(ctx) {
    +    val left = expression(ctx.left)
    +    val right = expression(ctx.right)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.AND =>
    +        And(left, right)
    +      case SqlBaseParser.OR =>
    +        Or(left, right)
    +    }
    --- End diff --
    
    That one's now merged


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198416483
  
    **[Test build #2651 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2651/consoleFull)** for PR 11557 at commit [`c5d0bcf`](https://github.com/apache/spark/commit/c5d0bcff0d41c1f302d842d6e111fa85a8baf74b).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57272248
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    +    assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b)))
    +    assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b))
    +  }
    +
    +  test("transform query spec") {
    +    val p = ScriptTransformation(Seq('a, 'b), "func", Seq.empty, table("e"), null)
    +    assertEqual("select transform(a, b) using 'func' from e where f < 10",
    +      p.copy(child = p.child.where('f < 10)))
    +    assertEqual("map(a, b) using 'func' as c, d from e",
    +      p.copy(output = Seq('c.string, 'd.string)))
    +    assertEqual("reduce(a, b) using 'func' as (c: int, d decimal(10, 0)) from e",
    +      p.copy(output = Seq('c.int, 'd.decimal(10, 0))))
    +  }
    +
    +  test("multi select query") {
    +    assertEqual(
    +      "from a select * select * where s < 10",
    --- End diff --
    
    do we have tests for multi-insert?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195882234
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193183240
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57272283
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    +    assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b)))
    +    assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b))
    +  }
    +
    +  test("transform query spec") {
    +    val p = ScriptTransformation(Seq('a, 'b), "func", Seq.empty, table("e"), null)
    +    assertEqual("select transform(a, b) using 'func' from e where f < 10",
    +      p.copy(child = p.child.where('f < 10)))
    +    assertEqual("map(a, b) using 'func' as c, d from e",
    +      p.copy(output = Seq('c.string, 'd.string)))
    +    assertEqual("reduce(a, b) using 'func' as (c: int, d decimal(10, 0)) from e",
    +      p.copy(output = Seq('c.int, 'd.decimal(10, 0))))
    +  }
    +
    +  test("multi select query") {
    +    assertEqual(
    +      "from a select * select * where s < 10",
    --- End diff --
    
    Is it allowed to use multi-select without the `insert`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55264084
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala ---
    @@ -331,8 +331,8 @@ class JoinSuite extends QueryTest with SharedSQLContext {
       }
     
       test("full outer join") {
    -    upperCaseData.where('N <= 4).registerTempTable("left")
    -    upperCaseData.where('N >= 3).registerTempTable("right")
    +    upperCaseData.where('N <= 4).registerTempTable("`left`")
    --- End diff --
    
    I think we should support it, but require quoting the names.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57149953
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/CatalystQlSuite.scala ---
    @@ -21,15 +21,18 @@ import org.apache.spark.sql.AnalysisException
     import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis._
     import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical._
     import org.apache.spark.unsafe.types.CalendarInterval
     
     class CatalystQlSuite extends PlanTest {
    -  val parser = new CatalystQl()
    +  val parser = CatalystSqlParser
    --- End diff --
    
    We can revert this line. The new ExpressionParserSuite/PlanParserSuite/TableIdentifierSuite contain all CatalystQlSuite tests or similar ones. So this is not really needed anymore.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202547496
  
    Merging this in master. Thanks.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279151
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    --- End diff --
    
    Oh I see, so although the parser accepts this syntax, we must detect this case in analyzer.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193182266
  
    **[Test build #52550 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52550/consoleFull)** for PR 11557 at commit [`06303bc`](https://github.com/apache/spark/commit/06303bcc2801b3f7520c3174a982e21056e86386).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279310
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    +    assertEqual("foo(*)", 'foo.function(1))
    --- End diff --
    
    That is currently normal behavior (also in the current parser) for count. I currently implemented this for all functions, and this should be for count only; I'll change this. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57166640
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala ---
    @@ -80,7 +80,7 @@ private[sql] class SessionState(ctx: SQLContext) {
       /**
        * Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
        */
    -  lazy val sqlParser: ParserInterface = new SparkQl(conf)
    +  lazy val sqlParser: ParserInterface = SparkSqlParser
    --- End diff --
    
    To confirm: is the antlr 3 code path still in use?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55368124
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala ---
    @@ -331,8 +331,8 @@ class JoinSuite extends QueryTest with SharedSQLContext {
       }
     
       test("full outer join") {
    -    upperCaseData.where('N <= 4).registerTempTable("left")
    -    upperCaseData.where('N >= 3).registerTempTable("right")
    +    upperCaseData.where('N <= 4).registerTempTable("`left`")
    --- End diff --
    
    That is what we currently do. You can use any keyword as long as you wrap it with backticks.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193367825
  
    **[Test build #52567 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52567/consoleFull)** for PR 11557 at commit [`c105f83`](https://github.com/apache/spark/commit/c105f83c12078adf2c3e3388f00797c43ee87dd5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-201084701
  
    LGTM except we parse `foo(*)` into `foo(1)`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198919390
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198919394
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53631/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195808206
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195911928
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279097
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    --- End diff --
    
    Do you mean `function(Symbol("foo.bar"))`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198136161
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55186195
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/ParseDriver.scala ---
    @@ -0,0 +1,205 @@
    +/*
    + * 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.parser.ng
    +
    +import org.antlr.v4.runtime._
    +import org.antlr.v4.runtime.atn.PredictionMode
    +import org.antlr.v4.runtime.misc.ParseCancellationException
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.sql.AnalysisException
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.expressions.Expression
    +import org.apache.spark.sql.catalyst.parser.ParserInterface
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
    +import org.apache.spark.util.Utils
    +
    +/**
    + * Base SQL parsing infrastructure.
    + */
    +abstract class AbstractSqlParser extends ParserInterface with Logging {
    +
    +  /** Creates Expression for a given SQL string. */
    +  override def parseExpression(sqlText: String): Expression = parse(sqlText) { parser =>
    +    astBuilder.visitSingleExpression(parser.singleExpression())
    +  }
    +
    +  /** Creates TableIdentifier for a given SQL string. */
    +  override def parseTableIdentifier(sqlText: String): TableIdentifier = parse(sqlText) { parser =>
    +    astBuilder.visitSingleTableIdentifier(parser.singleTableIdentifier())
    +  }
    +
    +  /** Creates LogicalPlan for a given SQL string. */
    +  override def parsePlan(sqlText: String): LogicalPlan = parse(sqlText) { parser =>
    +    astBuilder.visitSingleStatement(parser.singleStatement()) match {
    +      case plan: LogicalPlan => plan
    +      case _ => nativeCommand(sqlText)
    +    }
    +  }
    +
    +  /** Get the builder (visitor) which converts a ParseTree into a AST. */
    +  protected def astBuilder: AstBuilder
    +
    +  /** Create a native command, or fail when this is not supported. */
    +  protected def nativeCommand(sqlText: String): LogicalPlan = {
    +    throw new AnalysisException(s"Unsupported SQL statement:\n$sqlText")
    +  }
    +
    +  protected def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
    +    logInfo(s"Parsing command: $command")
    +
    +    val lexer = new SqlBaseLexer(new ANTLRNoCaseStringStream(command))
    +    lexer.removeErrorListeners()
    +    lexer.addErrorListener(ParseErrorListener)
    +
    +    val tokenStream = new CommonTokenStream(lexer)
    +    val parser = new SqlBaseParser(tokenStream)
    +    parser.addParseListener(PostProcessor)
    +    parser.removeErrorListeners()
    +    parser.addErrorListener(ParseErrorListener)
    +
    +    try {
    +      try {
    +        // first, try parsing with potentially faster SLL mode
    +        parser.getInterpreter.setPredictionMode(PredictionMode.SLL)
    +        toResult(parser)
    +      }
    +      catch {
    +        case e: ParseCancellationException =>
    +          // if we fail, parse with LL mode
    +          tokenStream.reset() // rewind input stream
    +          parser.reset()
    +
    +          // Try Again.
    +          parser.getInterpreter.setPredictionMode(PredictionMode.LL)
    +          toResult(parser)
    +      }
    +    }
    +    catch {
    +      case e: AnalysisException =>
    +        (e.line, e.startPosition) match {
    +          case (Some(line), Some(position)) =>
    --- End diff --
    
    This still a WIP. It shows users what went wrong.
    
    I want to move this into a separate exception.
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193426803
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193554765
  
    **[Test build #52613 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52613/consoleFull)** for PR 11557 at commit [`723edfb`](https://github.com/apache/spark/commit/723edfba11c40e832916d90b5d1453c926317022).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class ParseException(`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57271266
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    +    assertEqual("foo(*)", 'foo.function(1))
    --- End diff --
    
    why does the `*` parsed to `1`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57146948
  
    --- Diff: project/plugins.sbt ---
    @@ -23,3 +23,9 @@ libraryDependencies += "org.ow2.asm"  % "asm" % "5.0.3"
     libraryDependencies += "org.ow2.asm"  % "asm-commons" % "5.0.3"
     
     libraryDependencies += "org.antlr" % "antlr" % "3.5.2"
    +
    +
    +// TODO I am not sure we want such a dep.
    +resolvers += "simplytyped" at "http://simplytyped.github.io/repo/releases"
    --- End diff --
    
    Why we need it? Is it because we have to support both antlr 3 and 4 for now?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202429698
  
    **[Test build #54320 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54320/consoleFull)** for PR 11557 at commit [`6f1c535`](https://github.com/apache/spark/commit/6f1c535162397f01acf0405bdc80b8c4c141fc64).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198136155
  
    **[Test build #53480 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53480/consoleFull)** for PR 11557 at commit [`b87f2b8`](https://github.com/apache/spark/commit/b87f2b80e770f311326eeefb23e9d9b6ced62aa3).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #11557: [SPARK-13713][SQL] Migrate parser from ANTLR3 to ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r73654233
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1452 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import ParserUtils._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
    +    visit(ctx.dataType).asInstanceOf[DataType]
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Make sure we do not try to create a plan for a native command.
    +   */
    +  override def visitExecuteNativeCommand(ctx: ExecuteNativeCommandContext): LogicalPlan = null
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          throw new ParseException("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(string(pattern)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }
    +
    +      // Check for duplicate names.
    +      ctes.groupBy(_._1).filter(_._2.size > 1).foreach {
    +        case (name, _) =>
    +          throw new ParseException(
    +            s"Name '$name' is used for multiple common table expressions", ctx)
    +      }
    +
    +      With(query, ctes.toMap)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryResultClauses).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryResultClauses).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Create a partition specification map.
    +   */
    +  override def visitPartitionSpec(
    +      ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) {
    +    ctx.partitionVal.asScala.map { pVal =>
    +      val name = pVal.identifier.getText.toLowerCase
    +      val value = Option(pVal.constant).map(visitStringConstant)
    +      name -> value
    +    }.toMap
    +  }
    +
    +  /**
    +   * Create a partition specification map without optional values.
    +   */
    +  protected def visitNonOptionalPartitionSpec(
    +      ctx: PartitionSpecContext): Map[String, String] = withOrigin(ctx) {
    +    visitPartitionSpec(ctx).mapValues(_.orNull).map(identity)
    +  }
    +
    +  /**
    +   * Convert a constant of any type into a string. This is typically used in DDL commands, and its
    +   * main purpose is to prevent slight differences due to back to back conversions i.e.:
    +   * String -> Literal -> String.
    +   */
    +  protected def visitStringConstant(ctx: ConstantContext): String = withOrigin(ctx) {
    +    ctx match {
    +      case s: StringLiteralContext => createString(s)
    +      case o => o.getText
    +    }
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan. These
    +   * clauses determine the shape (ordering/partitioning/rows) of the query result.
    +   */
    +  private def withQueryResultClauses(
    +      ctx: QueryOrganizationContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
    +    val withOrder = if (
    +      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // ORDER BY ...
    +      Sort(order.asScala.map(visitSortItem), global = true, query)
    +    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ...
    +      Sort(sort.asScala.map(visitSortItem), global = false, query)
    +    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // DISTRIBUTE BY ...
    +      RepartitionByExpression(expressionList(distributeBy), query)
    +    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ... DISTRIBUTE BY ...
    +      Sort(
    +        sort.asScala.map(visitSortItem),
    +        global = false,
    +        RepartitionByExpression(expressionList(distributeBy), query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
    +      // CLUSTER BY ...
    +      val expressions = expressionList(clusterBy)
    +      Sort(
    +        expressions.map(SortOrder(_, Ascending)),
    +        global = false,
    +        RepartitionByExpression(expressions, query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // [EMPTY]
    +      query
    +    } else {
    +      throw new ParseException(
    +        "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
    +    }
    +
    +    // WINDOWS
    +    val withWindow = withOrder.optionalMap(windows)(withWindows)
    +
    +    // LIMIT
    +    withWindow.optional(limit) {
    +      Limit(typedVisit(limit), withWindow)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan using a query specification.
    +   */
    +  override def visitQuerySpecification(
    +      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
    +    val from = OneRowRelation.optional(ctx.fromClause) {
    +      visitFromClause(ctx.fromClause)
    +    }
    +    withQuerySpecification(ctx, from)
    +  }
    +
    +  /**
    +   * Add a query specification to a logical plan. The query specification is the core of the logical
    +   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
    +   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
    +   *
    +   * Note that query hints are ignored (both by the parser and the builder).
    +   */
    +  private def withQuerySpecification(
    +      ctx: QuerySpecificationContext,
    +      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // WHERE
    +    def filter(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = {
    +      Filter(expression(ctx), plan)
    +    }
    +
    +    // Expressions.
    +    val expressions = Option(namedExpressionSeq).toSeq
    +      .flatMap(_.namedExpression.asScala)
    +      .map(typedVisit[Expression])
    +
    +    // Create either a transform or a regular query.
    +    val specType = Option(kind).map(_.getType).getOrElse(SqlBaseParser.SELECT)
    +    specType match {
    +      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
    +        // Transform
    +
    +        // Add where.
    +        val withFilter = relation.optionalMap(where)(filter)
    +
    +        // Create the attributes.
    +        val (attributes, schemaLess) = if (colTypeList != null) {
    +          // Typed return columns.
    +          (createStructType(colTypeList).toAttributes, false)
    +        } else if (identifierSeq != null) {
    +          // Untyped return columns.
    +          val attrs = visitIdentifierSeq(identifierSeq).map { name =>
    +            AttributeReference(name, StringType, nullable = true)()
    +          }
    +          (attrs, false)
    +        } else {
    +          (Seq(AttributeReference("key", StringType)(),
    +            AttributeReference("value", StringType)()), true)
    +        }
    +
    +        // Create the transform.
    +        ScriptTransformation(
    +          expressions,
    +          string(script),
    +          attributes,
    +          withFilter,
    +          withScriptIOSchema(inRowFormat, recordWriter, outRowFormat, recordReader, schemaLess))
    +
    +      case SqlBaseParser.SELECT =>
    +        // Regular select
    +
    +        // Add lateral views.
    +        val withLateralView = ctx.lateralView.asScala.foldLeft(relation)(withGenerate)
    +
    +        // Add where.
    +        val withFilter = withLateralView.optionalMap(where)(filter)
    +
    +        // Add aggregation or a project.
    +        val namedExpressions = expressions.map {
    +          case e: NamedExpression => e
    +          case e: Expression => UnresolvedAlias(e)
    +        }
    +        val withProject = if (aggregation != null) {
    +          withAggregation(aggregation, namedExpressions, withFilter)
    +        } else if (namedExpressions.nonEmpty) {
    +          Project(namedExpressions, withFilter)
    +        } else {
    +          withFilter
    +        }
    +
    +        // Having
    +        val withHaving = withProject.optional(having) {
    +          // Note that we added a cast to boolean. If the expression itself is already boolean,
    +          // the optimizer will get rid of the unnecessary cast.
    +          Filter(Cast(expression(having), BooleanType), withProject)
    +        }
    +
    +        // Distinct
    +        val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) {
    +          Distinct(withHaving)
    +        } else {
    +          withHaving
    +        }
    +
    +        // Window
    +        withDistinct.optionalMap(windows)(withWindows)
    +    }
    +  }
    +
    +  /**
    +   * Create a (Hive based) [[ScriptInputOutputSchema]].
    +   */
    +  protected def withScriptIOSchema(
    +      inRowFormat: RowFormatContext,
    +      recordWriter: Token,
    +      outRowFormat: RowFormatContext,
    +      recordReader: Token,
    +      schemaLess: Boolean): ScriptInputOutputSchema = null
    +
    +  /**
    +   * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
    +   * separated) relations here, these get converted into a single plan by condition-less inner join.
    +   */
    +  override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
    +    val from = ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None))
    +    ctx.lateralView.asScala.foldLeft(from)(withGenerate)
    +  }
    +
    +  /**
    +   * Connect two queries by a Set operator.
    +   *
    +   * Supported Set operators are:
    +   * - UNION [DISTINCT]
    +   * - UNION ALL
    +   * - EXCEPT [DISTINCT]
    +   * - INTERSECT [DISTINCT]
    +   */
    +  override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
    +    val left = plan(ctx.left)
    +    val right = plan(ctx.right)
    +    val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.UNION if all =>
    +        Union(left, right)
    +      case SqlBaseParser.UNION =>
    +        Distinct(Union(left, right))
    +      case SqlBaseParser.INTERSECT if all =>
    +        throw new ParseException("INTERSECT ALL is not supported.", ctx)
    +      case SqlBaseParser.INTERSECT =>
    +        Intersect(left, right)
    +      case SqlBaseParser.EXCEPT if all =>
    +        throw new ParseException("EXCEPT ALL is not supported.", ctx)
    +      case SqlBaseParser.EXCEPT =>
    +        Except(left, right)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[WithWindowDefinition]] operator to a logical plan.
    +   */
    +  private def withWindows(
    +      ctx: WindowsContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Collect all window specifications defined in the WINDOW clause.
    +    val baseWindowMap = ctx.namedWindow.asScala.map {
    +      wCtx =>
    +        (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
    +    }.toMap
    +
    +    // Handle cases like
    +    // window w1 as (partition by p_mfgr order by p_name
    +    //               range between 2 preceding and 2 following),
    +    //        w2 as w1
    +    val windowMapView = baseWindowMap.mapValues {
    +      case WindowSpecReference(name) =>
    +        baseWindowMap.get(name) match {
    +          case Some(spec: WindowSpecDefinition) =>
    +            spec
    +          case Some(ref) =>
    +            throw new ParseException(s"Window reference '$name' is not a window specification", ctx)
    +          case None =>
    +            throw new ParseException(s"Cannot resolve window reference '$name'", ctx)
    +        }
    +      case spec: WindowSpecDefinition => spec
    +    }
    +
    +    // Note that mapValues creates a view instead of materialized map. We force materialization by
    +    // mapping over identity.
    +    WithWindowDefinition(windowMapView.map(identity), query)
    +  }
    +
    +  /**
    +   * Add an [[Aggregate]] to a logical plan.
    +   */
    +  private def withAggregation(
    +      ctx: AggregationContext,
    +      selectExpressions: Seq[NamedExpression],
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    val groupByExpressions = expressionList(groupingExpressions)
    +
    +    if (GROUPING != null) {
    +      // GROUP BY .... GROUPING SETS (...)
    +      val expressionMap = groupByExpressions.zipWithIndex.toMap
    +      val numExpressions = expressionMap.size
    +      val mask = (1 << numExpressions) - 1
    +      val masks = ctx.groupingSet.asScala.map {
    +        _.expression.asScala.foldLeft(mask) {
    +          case (bitmap, eCtx) =>
    +            // Find the index of the expression.
    +            val e = typedVisit[Expression](eCtx)
    +            val index = expressionMap.find(_._1.semanticEquals(e)).map(_._2).getOrElse(
    +              throw new ParseException(
    +                s"$e doesn't show up in the GROUP BY list", ctx))
    +            // 0 means that the column at the given index is a grouping column, 1 means it is not,
    +            // so we unset the bit in bitmap.
    +            bitmap & ~(1 << (numExpressions - 1 - index))
    +        }
    +      }
    +      GroupingSets(masks, groupByExpressions, query, selectExpressions)
    +    } else {
    +      // GROUP BY .... (WITH CUBE | WITH ROLLUP)?
    +      val mappedGroupByExpressions = if (CUBE != null) {
    +        Seq(Cube(groupByExpressions))
    +      } else if (ROLLUP != null) {
    +        Seq(Rollup(groupByExpressions))
    +      } else {
    +        groupByExpressions
    +      }
    +      Aggregate(mappedGroupByExpressions, selectExpressions, query)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[Generate]] (Lateral View) to a logical plan.
    +   */
    +  private def withGenerate(
    +      query: LogicalPlan,
    +      ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
    +    val expressions = expressionList(ctx.expression)
    +
    +    // Create the generator.
    +    val generator = ctx.qualifiedName.getText.toLowerCase match {
    +      case "explode" if expressions.size == 1 =>
    +        Explode(expressions.head)
    +      case "json_tuple" =>
    +        JsonTuple(expressions)
    +      case other =>
    +        withGenerator(other, expressions, ctx)
    +    }
    +
    +    Generate(
    +      generator,
    +      join = true,
    +      outer = ctx.OUTER != null,
    +      Some(ctx.tblName.getText.toLowerCase),
    +      ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply),
    +      query)
    +  }
    +
    +  /**
    +   * Create a [[Generator]]. Override this method in order to support custom Generators.
    +   */
    +  protected def withGenerator(
    +      name: String,
    +      expressions: Seq[Expression],
    +      ctx: LateralViewContext): Generator = {
    +    throw new ParseException(s"Generator function '$name' is not supported", ctx)
    +  }
    +
    +  /**
    +   * Create a joins between two or more logical plans.
    +   */
    +  override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) {
    +    /** Build a join between two plans. */
    +    def join(ctx: JoinRelationContext, left: LogicalPlan, right: LogicalPlan): Join = {
    +      val baseJoinType = ctx.joinType match {
    +        case null => Inner
    +        case jt if jt.FULL != null => FullOuter
    +        case jt if jt.SEMI != null => LeftSemi
    +        case jt if jt.LEFT != null => LeftOuter
    +        case jt if jt.RIGHT != null => RightOuter
    +        case _ => Inner
    +      }
    +
    +      // Resolve the join type and join condition
    +      val (joinType, condition) = Option(ctx.joinCriteria) match {
    +        case Some(c) if c.USING != null =>
    +          val columns = c.identifier.asScala.map { column =>
    +            UnresolvedAttribute.quoted(column.getText)
    +          }
    +          (UsingJoin(baseJoinType, columns), None)
    +        case Some(c) if c.booleanExpression != null =>
    +          (baseJoinType, Option(expression(c.booleanExpression)))
    +        case None if ctx.NATURAL != null =>
    +          (NaturalJoin(baseJoinType), None)
    +        case None =>
    +          (baseJoinType, None)
    +      }
    +      Join(left, right, joinType, condition)
    +    }
    +
    +    // Handle all consecutive join clauses. ANTLR produces a right nested tree in which the the
    +    // first join clause is at the top. However fields of previously referenced tables can be used
    +    // in following join clauses. The tree needs to be reversed in order to make this work.
    +    var result = plan(ctx.left)
    +    var current = ctx
    +    while (current != null) {
    +      current.right match {
    +        case right: JoinRelationContext =>
    +          result = join(current, result, plan(right.left))
    +          current = right
    +        case right =>
    +          result = join(current, result, plan(right))
    +          current = null
    +      }
    +    }
    +    result
    +  }
    +
    +  /**
    +   * Add a [[Sample]] to a logical plan.
    +   *
    +   * This currently supports the following sampling methods:
    +   * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
    +   * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
    +   * are defined as a number between 0 and 100.
    +   * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
    +   */
    +  private def withSample(ctx: SampleContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Create a sampled plan if we need one.
    +    def sample(fraction: Double): Sample = {
    +      // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
    +      // function takes X PERCENT as the input and the range of X is [0, 100], we need to
    +      // adjust the fraction.
    +      val eps = RandomSampler.roundingEpsilon
    +      assert(fraction >= 0.0 - eps && fraction <= 1.0 + eps,
    +        s"Sampling fraction ($fraction) must be on interval [0, 1]",
    +        ctx)
    +      Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)(true)
    +    }
    +
    +    ctx.sampleType.getType match {
    +      case SqlBaseParser.ROWS =>
    +        Limit(expression(ctx.expression), query)
    +
    +      case SqlBaseParser.PERCENTLIT =>
    +        val fraction = ctx.percentage.getText.toDouble
    +        sample(fraction / 100.0d)
    +
    +      case SqlBaseParser.BUCKET if ctx.ON != null =>
    +        throw new ParseException("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx)
    +
    +      case SqlBaseParser.BUCKET =>
    +        sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a sub-query.
    +   */
    +  override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryNoWith)
    +  }
    +
    +  /**
    +   * Create an un-aliased table reference. This is typically used for top-level table references,
    +   * for example:
    +   * {{{
    +   *   INSERT INTO db.tbl2
    +   *   TABLE db.tbl1
    +   * }}}
    +   */
    +  override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None)
    +  }
    +
    +  /**
    +   * Create an aliased table reference. This is typically used in FROM clauses.
    +   */
    +  override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
    +    val table = UnresolvedRelation(
    +      visitTableIdentifier(ctx.tableIdentifier),
    +      Option(ctx.identifier).map(_.getText))
    +    table.optionalMap(ctx.sample)(withSample)
    +  }
    +
    +  /**
    +   * Create an inline table (a virtual table in Hive parlance).
    +   */
    +  override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
    +    // Get the backing expressions.
    +    val expressions = ctx.expression.asScala.map { eCtx =>
    +      val e = expression(eCtx)
    +      assert(e.foldable, "All expressions in an inline table must be constants.", eCtx)
    +      e
    +    }
    +
    +    // Validate and evaluate the rows.
    +    val (structType, structConstructor) = expressions.head.dataType match {
    --- End diff --
    
    what if it's not castable? and what if the first one is int and the second one is long? Logically we should cast the first one to long, how does other database handle it?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57165625
  
    --- Diff: project/plugins.sbt ---
    @@ -23,3 +23,9 @@ libraryDependencies += "org.ow2.asm"  % "asm" % "5.0.3"
     libraryDependencies += "org.ow2.asm"  % "asm-commons" % "5.0.3"
     
     libraryDependencies += "org.antlr" % "antlr" % "3.5.2"
    +
    +
    +// TODO I am not sure we want such a dep.
    +resolvers += "simplytyped" at "http://simplytyped.github.io/repo/releases"
    --- End diff --
    
    Is it the standard way to run antlr 4 in SBT? Just curious about how hard it would be if we don't use this plugin.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-199785656
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53764/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279658
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    +    assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b)))
    +    assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b))
    +  }
    +
    +  test("transform query spec") {
    +    val p = ScriptTransformation(Seq('a, 'b), "func", Seq.empty, table("e"), null)
    +    assertEqual("select transform(a, b) using 'func' from e where f < 10",
    +      p.copy(child = p.child.where('f < 10)))
    +    assertEqual("map(a, b) using 'func' as c, d from e",
    +      p.copy(output = Seq('c.string, 'd.string)))
    +    assertEqual("reduce(a, b) using 'func' as (c: int, d decimal(10, 0)) from e",
    +      p.copy(output = Seq('c.int, 'd.decimal(10, 0))))
    +  }
    +
    +  test("multi select query") {
    +    assertEqual(
    +      "from a select * select * where s < 10",
    --- End diff --
    
    It is but it does not make much sense. Hive supports this (It returns multiple result sets).
    
    A multi-insert will then revert to a union all; so the queries should all have the same number of ouputs and similar output types.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55185301
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1128 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.sql.AnalysisException
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ParseUtils
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import AstBuilder._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          notSupported("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(unquote(pattern.getText)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }.toMap
    +      With(query, ctes)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryOrganization).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryOrganization).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).toSeq.flatMap {
    +      _.partitionVal.asScala.map {
    +        pVal => (pVal.identifier.getText, Option(pVal.constant).map(c => unquote(c.getText)))
    +      }
    +    }.toMap
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan.
    +   */
    +  private def withQueryOrganization(
    +      ctx: QueryOrganizationContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
    +    val withOrder = if (
    +      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // ORDER BY ...
    +      Sort(order.asScala.map(visitSortItem), global = true, query)
    +    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ...
    +      Sort(sort.asScala.map(visitSortItem), global = false, query)
    +    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // DISTRIBUTE BY ...
    +      RepartitionByExpression(expressionList(distributeBy), query)
    +    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ... DISTRIBUTE BY ...
    +      Sort(
    +        sort.asScala.map(visitSortItem),
    +        global = false,
    +        RepartitionByExpression(expressionList(distributeBy), query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
    +      // CLUSTER BY ...
    +      val expressions = expressionList(clusterBy)
    +      Sort(
    +        expressions.map(SortOrder(_, Ascending)),
    +        global = false,
    +        RepartitionByExpression(expressions, query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // [EMPTY]
    +      query
    +    } else {
    +      notSupported("Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
    +    }
    +
    +    // LIMIT
    +    val withLimit = withOrder.optional(limit) {
    +      Limit(typedVisit(limit), withOrder)
    +    }
    +
    +    // WINDOWS
    +    withLimit.optionalMap(windows)(withWindows)
    +  }
    +
    +  /**
    +   * Create a logical plan using a query specification.
    +   */
    +  override def visitQuerySpecification(
    +      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
    +    val from = OneRowRelation.optional(ctx.fromClause) {
    +      visitFromClause(ctx.fromClause)
    +    }
    +    withQuerySpecification(ctx, from)
    +  }
    +
    +  /**
    +   * Add a query specification to a logical plan. The query specification is the core of the logical
    +   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
    +   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
    +   *
    +   * Note that query hints are ignored (both by the parser and the builder).
    +   */
    +  private def withQuerySpecification(
    +      ctx: QuerySpecificationContext,
    +      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // WHERE
    +    val withFilter = relation.optional(where) {
    +      Filter(expression(where), relation)
    +    }
    +
    +    // Expressions.
    +    val expressions = namedExpression.asScala.map(visit).map {
    +      case e: Expression => UnresolvedAlias(e)
    +    }
    +
    +    // Create either a transform or a regular query.
    +    kind.getType match {
    +      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
    +        // Transform
    +
    +        // Create the attributes.
    +        val attributes = if (colTypeList != null) {
    +          // Typed return columns.
    +          visitColTypeList(colTypeList).toAttributes
    +        } else if (columnAliasList != null) {
    +          // Untyped return columns.
    +          visitColumnAliasList(columnAliasList).map { name =>
    +            AttributeReference(name, StringType, nullable = true)()
    +          }
    +        } else {
    +          Seq.empty
    +        }
    +
    +        // Create the transform.
    +        ScriptTransformation(
    +          expressions,
    +          unquote(script.getText),
    +          attributes,
    +          withFilter,
    +          withScriptIOSchema(inRowFormat, outRowFormat, outRecordReader))
    +
    +      case SqlBaseParser.SELECT =>
    +        // Regular select
    +
    +        // Add lateral views.
    +        val withLateralView = ctx.lateralView.asScala.foldLeft(withFilter)(withGenerate)
    +
    +        // Add aggregation with having or a project.
    +        val withProject = if (aggregation != null) {
    +          withAggregation(aggregation, expressions, withLateralView).optionalMap(having) {
    +            case (h, p) => Filter(expression(h), p)
    --- End diff --
    
    Note to self: Group By and Having clause should not be pipelined, a user is allowed to define a global aggregate and filter that...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193704913
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/52646/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57269065
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    --- End diff --
    
    Is this allowed? I tried it on PostgreSQL, it throws exception.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-199448401
  
    Let me know if we should start reviewing this.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57185205
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1450 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ParseUtils
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import AstBuilder._
    +  import ParseUtils._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
    +    visit(ctx.dataType).asInstanceOf[DataType]
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          throw new ParseException("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(unescapeSQLString(pattern.getText)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }
    +
    +      // Check for duplicate names.
    +      ctes.groupBy(_._1).filter(_._2.size > 1).foreach {
    +        case (name, _) =>
    +          throw new ParseException(
    +            s"Name '$name' is used for multiple common table expressions", ctx)
    +      }
    +
    +      With(query, ctes.toMap)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryOrganization).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryOrganization).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).toSeq.flatMap {
    +      _.partitionVal.asScala.map {
    +        pVal =>
    +          val name = pVal.identifier.getText
    +          val value = Option(pVal.constant).map(c => expression(c).toString)
    +          name -> value
    +      }
    +    }.toMap
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan.
    +   */
    +  private def withQueryOrganization(
    --- End diff --
    
    `withQueryClauses`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195911883
  
    **[Test build #53022 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53022/consoleFull)** for PR 11557 at commit [`460ef0d`](https://github.com/apache/spark/commit/460ef0d77a366602c89d87e9518fc94a8b721dd3).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193521621
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/52594/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57278901
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    --- End diff --
    
    When referencing a function defined in another database? (Not quite sure about this.)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279846
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    +    assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b)))
    +    assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b))
    +  }
    +
    +  test("transform query spec") {
    +    val p = ScriptTransformation(Seq('a, 'b), "func", Seq.empty, table("e"), null)
    +    assertEqual("select transform(a, b) using 'func' from e where f < 10",
    +      p.copy(child = p.child.where('f < 10)))
    +    assertEqual("map(a, b) using 'func' as c, d from e",
    +      p.copy(output = Seq('c.string, 'd.string)))
    +    assertEqual("reduce(a, b) using 'func' as (c: int, d decimal(10, 0)) from e",
    +      p.copy(output = Seq('c.int, 'd.decimal(10, 0))))
    +  }
    +
    +  test("multi select query") {
    +    assertEqual(
    +      "from a select * select * where s < 10",
    +      table("a").select(star()).unionAll(table("a").where('s < 10).select(star())))
    +    intercept(
    +      "from a select * select * from x where a.s < 10",
    +      "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements")
    +    assertEqual(
    +      "from a insert into tbl1 select * insert into tbl2 select * where s < 10",
    +      table("a").select(star()).insertInto("tbl1").unionAll(
    +        table("a").where('s < 10).select(star()).insertInto("tbl2")))
    +  }
    +
    +  test("query organization") {
    +    // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows
    +    val baseSql = "select * from t"
    +    val basePlan = table("t").select(star())
    +
    +    val ws = Map("w1" -> WindowSpecDefinition(Seq.empty, Seq.empty, UnspecifiedFrame))
    +    val limitWindowClauses = Seq(
    +      ("", (p: LogicalPlan) => p),
    +      (" limit 10", (p: LogicalPlan) => p.limit(10)),
    +      (" window w1 as ()", (p: LogicalPlan) => WithWindowDefinition(ws, p)),
    +      (" window w1 as () limit 10", (p: LogicalPlan) => WithWindowDefinition(ws, p).limit(10))
    +    )
    +
    +    val orderSortDistrClusterClauses = Seq(
    +      ("", basePlan),
    +      (" order by a, b desc", basePlan.orderBy('a.asc, 'b.desc)),
    +      (" sort by a, b desc", basePlan.sortBy('a.asc, 'b.desc)),
    +      (" distribute by a, b", basePlan.distribute('a, 'b)),
    +      (" distribute by a sort by b", basePlan.distribute('a).sortBy('b.asc)),
    +      (" cluster by a, b", basePlan.distribute('a, 'b).sortBy('a.asc, 'b.asc))
    +    )
    +
    +    orderSortDistrClusterClauses.foreach {
    +      case (s1, p1) =>
    +        limitWindowClauses.foreach {
    +          case (s2, pf2) =>
    +            assertEqual(baseSql + s1 + s2, pf2(p1))
    +        }
    +    }
    +
    +    val msg = "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported"
    +    intercept(s"$baseSql order by a sort by a", msg)
    +    intercept(s"$baseSql cluster by a distribute by a", msg)
    +    intercept(s"$baseSql order by a cluster by a", msg)
    +    intercept(s"$baseSql order by a distribute by a", msg)
    +  }
    +
    +  test("insert into") {
    +    val sql = "select * from t"
    +    val plan = table("t").select(star())
    +    def insert(
    +        partition: Map[String, Option[String]],
    +        overwrite: Boolean = false,
    +        ifNotExists: Boolean = false): LogicalPlan =
    +      InsertIntoTable(table("s"), partition, plan, overwrite, ifNotExists)
    +
    +    // Single inserts
    +    assertEqual(s"insert overwrite table s $sql",
    +      insert(Map.empty, overwrite = true))
    +    assertEqual(s"insert overwrite table s if not exists $sql",
    +      insert(Map.empty, overwrite = true, ifNotExists = true))
    +    assertEqual(s"insert into s $sql",
    +      insert(Map.empty))
    +    assertEqual(s"insert into table s partition (c = 'd', e = 1) $sql",
    +      insert(Map("c" -> Option("d"), "e" -> Option("1"))))
    +    assertEqual(s"insert overwrite table s partition (c = 'd', x) if not exists $sql",
    +      insert(Map("c" -> Option("d"), "x" -> None), overwrite = true, ifNotExists = true))
    +
    +    // Multi insert
    +    val plan2 = table("t").where('x > 5).select(star())
    +    assertEqual("from t insert into s select * limit 1 insert into u select * where x > 5",
    +      InsertIntoTable(
    +        table("s"), Map.empty, plan.limit(1), overwrite = false, ifNotExists = false).unionAll(
    +        InsertIntoTable(
    +          table("u"), Map.empty, plan2, overwrite = false, ifNotExists = false)))
    +  }
    +
    +  test("aggregation") {
    +    val sql = "select a, b, sum(c) as c from d group by a, b"
    +
    +    // Normal
    +    assertEqual(sql, table("d").groupBy('a, 'b)('a, 'b, 'sum.function('c).as("c")))
    +
    +    // Cube
    +    assertEqual(s"$sql with cube",
    +      table("d").groupBy(Cube(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
    +
    +    // Rollup
    +    assertEqual(s"$sql with rollup",
    +      table("d").groupBy(Rollup(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
    +
    +    // Grouping Sets
    +    assertEqual(s"$sql grouping sets((a, b), (a), ())",
    +      GroupingSets(Seq(0, 1, 3), Seq('a, 'b), table("d"), Seq('a, 'b, 'sum.function('c).as("c"))))
    +    intercept(s"$sql grouping sets((a, b), (c), ())",
    +      "c doesn't show up in the GROUP BY list")
    +  }
    +
    +  test("limit") {
    +    val sql = "select * from t"
    +    val plan = table("t").select(star())
    +    assertEqual(s"$sql limit 10", plan.limit(10))
    +    assertEqual(s"$sql limit cast(9 / 4 as int)", plan.limit(Cast(Literal(9) / 4, IntegerType)))
    +  }
    +
    +  test("window spec") {
    +    // Note that WindowSpecs are testing in the ExpressionParserSuite
    +    val sql = "select * from t"
    +    val plan = table("t").select(star())
    +    val spec = WindowSpecDefinition(Seq('a, 'b), Seq('c.asc),
    +      SpecifiedWindowFrame(RowFrame, ValuePreceding(1), ValueFollowing(1)))
    +
    +    // Test window resolution.
    +    val ws1 = Map("w1" -> spec, "w2" -> spec, "w3" -> spec)
    +    assertEqual(
    +      s"""$sql
    +         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
    +         |       w2 as w1,
    +         |       w3 as w1""".stripMargin,
    +      WithWindowDefinition(ws1, plan))
    +
    +    // Fail with no reference.
    +    intercept(s"$sql window w2 as w1", "Cannot resolve window reference 'w1'")
    +
    +    // Fail when resolved reference is not a window spec.
    +    intercept(
    +      s"""$sql
    +         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
    +         |       w2 as w1,
    +         |       w3 as w2""".stripMargin,
    +      "Window reference 'w2' is not a window specification"
    +    )
    +  }
    +
    +  test("lateral view") {
    +    // Single lateral view
    +    assertEqual(
    +      "select * from t lateral view explode(x) expl as x",
    +      table("t")
    +        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
    +        .select(star()))
    +
    +    // Multiple lateral views
    +    assertEqual(
    +      """select *
    +        |from t
    +        |lateral view explode(x) expl as x
    +        |lateral view outer json_tuple(x, y) jtup q, z""".stripMargin,
    +      table("t")
    +        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
    +        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = true, Some("jtup"), Seq("q", "z"))
    +        .select(star()))
    +
    +    // Multi-Insert lateral views.
    +    val from = table("t1").generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
    +    assertEqual(
    +      """from t1
    +        |lateral view explode(x) expl as x
    +        |insert into t2
    +        |select *
    +        |lateral view json_tuple(x, y) jtup q, z
    +        |insert into t3
    +        |select *
    +        |where s < 10
    +      """.stripMargin,
    +      Union(from
    +        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = false, Some("jtup"), Seq("q", "z"))
    +        .select(star())
    +        .insertInto("t2"),
    +        from.where('s < 10).select(star()).insertInto("t3")))
    +
    +    // Unsupported generator.
    +    intercept(
    +      "select * from t lateral view posexplode(x) posexpl as x, y",
    +      "Generator function 'posexplode' is not supported")
    +  }
    +
    +  test("joins") {
    +    val testUnconditionalJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t as tt $sql u",
    +        table("t").as("tt").join(table("u"), jt, None).select(star()))
    +    }
    +    val testConditionalJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t $sql u as uu on a = b",
    +        table("t").join(table("u").as("uu"), jt, Option('a === 'b)).select(star()))
    +    }
    +    val testNaturalJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t tt natural $sql u as uu",
    +        table("t").as("tt").join(table("u").as("uu"), NaturalJoin(jt), None).select(star()))
    +    }
    +    val testUsingJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t $sql u using(a, b)",
    +        table("t").join(table("u"), UsingJoin(jt, Seq('a.attr, 'b.attr)), None).select(star()))
    +    }
    +    val testAll = Seq(testUnconditionalJoin, testConditionalJoin, testNaturalJoin, testUsingJoin)
    +
    +    def test(sql: String, jt: JoinType, tests: Seq[(String, JoinType) => Unit]): Unit = {
    +      tests.foreach(_(sql, jt))
    +    }
    +    test("cross join", Inner, Seq(testUnconditionalJoin))
    +    test(",", Inner, Seq(testUnconditionalJoin))
    +    test("join", Inner, testAll)
    +    test("inner join", Inner, testAll)
    +    test("left join", LeftOuter, testAll)
    +    test("left outer join", LeftOuter, testAll)
    +    test("right join", RightOuter, testAll)
    +    test("right outer join", RightOuter, testAll)
    +    test("full join", FullOuter, testAll)
    +    test("full outer join", FullOuter, testAll)
    +  }
    +
    +  test("sampled relations") {
    +    val sql = "select * from t"
    +    assertEqual(s"$sql tablesample(100 rows)",
    +      table("t").limit(100).select(star()))
    +    assertEqual(s"$sql as x tablesample(43 percent)",
    +      Sample(0, .43d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
    +    assertEqual(s"$sql as x tablesample(bucket 4 out of 10)",
    +      Sample(0, .4d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
    +    intercept(s"$sql as x tablesample(bucket 4 out of 10 on x)",
    +      "TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported")
    +    intercept(s"$sql as x tablesample(bucket 11 out of 10)",
    +      s"Sampling fraction (${11.0/10.0}) must be on interval [0, 1]")
    +  }
    +
    +  test("sub-query") {
    +    val plan = table("t0").select('id)
    +    assertEqual("select id from (t0)", plan)
    +    assertEqual("select id from ((((((t0))))))", plan)
    +    assertEqual(
    +      "(select * from t1) union distinct (select * from t2)",
    +      Distinct(table("t1").select(star()).unionAll(table("t2").select(star()))))
    +    assertEqual(
    +      "select * from ((select * from t1) union (select * from t2)) t",
    +      Distinct(
    +        table("t1").select(star()).unionAll(table("t2").select(star()))).as("t").select(star()))
    +    assertEqual(
    +      """select  id
    +        |from (((select id from t0)
    +        |       union all
    +        |       (select  id from t0))
    +        |      union all
    +        |      (select id from t0)) as u_1
    +      """.stripMargin,
    +      plan.unionAll(plan).unionAll(plan).as("u_1").select('id))
    +  }
    +
    +  test("scalar sub-query") {
    +    assertEqual(
    +      "select (select max(b) from s) ss from t",
    +      table("t").select(ScalarSubquery(table("s").select('max.function('b))).as("ss")))
    +    assertEqual(
    +      "select * from t where a = (select b from s)",
    +      table("t").where('a === ScalarSubquery(table("s").select('b))).select(star()))
    +    assertEqual(
    +      "select g from t group by g having a > (select b from s)",
    +      table("t").groupBy('g)('g).where('a > ScalarSubquery(table("s").select('b))))
    +  }
    +
    +  test("table reference") {
    +    assertEqual("table t", table("t"))
    +    assertEqual("table d.t", table("d", "t"))
    +  }
    +
    +  test("inline table") {
    +    assertEqual("values 1, 2, 3, 4", LocalRelation.fromExternalRows(
    --- End diff --
    
        insert into temp_tbl
        values (1, 'a'), (2, 'b'), (3, 'c') as x(a, b)



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279657
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    +    assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b)))
    +    assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b))
    +  }
    +
    +  test("transform query spec") {
    +    val p = ScriptTransformation(Seq('a, 'b), "func", Seq.empty, table("e"), null)
    +    assertEqual("select transform(a, b) using 'func' from e where f < 10",
    +      p.copy(child = p.child.where('f < 10)))
    +    assertEqual("map(a, b) using 'func' as c, d from e",
    +      p.copy(output = Seq('c.string, 'd.string)))
    +    assertEqual("reduce(a, b) using 'func' as (c: int, d decimal(10, 0)) from e",
    +      p.copy(output = Seq('c.int, 'd.decimal(10, 0))))
    +  }
    +
    +  test("multi select query") {
    +    assertEqual(
    +      "from a select * select * where s < 10",
    --- End diff --
    
    It is but it does not make much sense. Hive supports this (It returned multiple result sets).
    
    A multi-insert will then revert to a union all; so the queries should all have the same number of ouputs and similar output types.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279436
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    --- End diff --
    
    So this syntax should be allowed, but we need to fail analysis. However, we will convert both `WHERE` and `HAVING` to `Filter`, it's a bit hard to distinguish them during analysis.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202494895
  
    **[Test build #54320 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/54320/consoleFull)** for PR 11557 at commit [`6f1c535`](https://github.com/apache/spark/commit/6f1c535162397f01acf0405bdc80b8c4c141fc64).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202495312
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-200385241
  
    Overall LGTM(I haven't looked into every parser rule though), it's much more readable than the antlr 3 version! Could you upload the generated java files somewhere and link them at this PR? I think they will be helpful for detailed code review, thanks!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by zsxwing <gi...@git.apache.org>.
Github user zsxwing commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202555774
  
    Seems maven stuff is not set correctly?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r55195953
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1128 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.Logging
    +import org.apache.spark.sql.AnalysisException
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ParseUtils
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.catalyst.trees.CurrentOrigin
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import AstBuilder._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          notSupported("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(unquote(pattern.getText)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }.toMap
    +      With(query, ctes)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryOrganization).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryOrganization).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).toSeq.flatMap {
    +      _.partitionVal.asScala.map {
    +        pVal => (pVal.identifier.getText, Option(pVal.constant).map(c => unquote(c.getText)))
    +      }
    +    }.toMap
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan.
    +   */
    +  private def withQueryOrganization(
    +      ctx: QueryOrganizationContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
    +    val withOrder = if (
    +      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // ORDER BY ...
    +      Sort(order.asScala.map(visitSortItem), global = true, query)
    +    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ...
    +      Sort(sort.asScala.map(visitSortItem), global = false, query)
    +    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // DISTRIBUTE BY ...
    +      RepartitionByExpression(expressionList(distributeBy), query)
    +    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ... DISTRIBUTE BY ...
    +      Sort(
    +        sort.asScala.map(visitSortItem),
    +        global = false,
    +        RepartitionByExpression(expressionList(distributeBy), query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
    +      // CLUSTER BY ...
    +      val expressions = expressionList(clusterBy)
    +      Sort(
    +        expressions.map(SortOrder(_, Ascending)),
    +        global = false,
    +        RepartitionByExpression(expressions, query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // [EMPTY]
    +      query
    +    } else {
    +      notSupported("Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
    +    }
    +
    +    // LIMIT
    +    val withLimit = withOrder.optional(limit) {
    +      Limit(typedVisit(limit), withOrder)
    +    }
    +
    +    // WINDOWS
    +    withLimit.optionalMap(windows)(withWindows)
    +  }
    +
    +  /**
    +   * Create a logical plan using a query specification.
    +   */
    +  override def visitQuerySpecification(
    +      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
    +    val from = OneRowRelation.optional(ctx.fromClause) {
    +      visitFromClause(ctx.fromClause)
    +    }
    +    withQuerySpecification(ctx, from)
    +  }
    +
    +  /**
    +   * Add a query specification to a logical plan. The query specification is the core of the logical
    +   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
    +   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
    +   *
    +   * Note that query hints are ignored (both by the parser and the builder).
    +   */
    +  private def withQuerySpecification(
    +      ctx: QuerySpecificationContext,
    +      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // WHERE
    +    val withFilter = relation.optional(where) {
    +      Filter(expression(where), relation)
    +    }
    +
    +    // Expressions.
    +    val expressions = namedExpression.asScala.map(visit).map {
    +      case e: Expression => UnresolvedAlias(e)
    +    }
    +
    +    // Create either a transform or a regular query.
    +    kind.getType match {
    +      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
    +        // Transform
    +
    +        // Create the attributes.
    +        val attributes = if (colTypeList != null) {
    +          // Typed return columns.
    +          visitColTypeList(colTypeList).toAttributes
    +        } else if (columnAliasList != null) {
    +          // Untyped return columns.
    +          visitColumnAliasList(columnAliasList).map { name =>
    +            AttributeReference(name, StringType, nullable = true)()
    +          }
    +        } else {
    +          Seq.empty
    +        }
    +
    +        // Create the transform.
    +        ScriptTransformation(
    +          expressions,
    +          unquote(script.getText),
    +          attributes,
    +          withFilter,
    +          withScriptIOSchema(inRowFormat, outRowFormat, outRecordReader))
    +
    +      case SqlBaseParser.SELECT =>
    +        // Regular select
    +
    +        // Add lateral views.
    +        val withLateralView = ctx.lateralView.asScala.foldLeft(withFilter)(withGenerate)
    +
    +        // Add aggregation with having or a project.
    +        val withProject = if (aggregation != null) {
    +          withAggregation(aggregation, expressions, withLateralView).optionalMap(having) {
    +            case (h, p) => Filter(expression(h), p)
    +          }
    +        } else {
    +          Project(expressions, withLateralView)
    +        }
    +
    +        // Distinct
    +        val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) {
    +          Distinct(withProject)
    +        } else {
    +          withProject
    +        }
    +
    +        // Window
    +        withDistinct.optionalMap(windows)(withWindows)
    +    }
    +  }
    +
    +  /**
    +   * Create a (Hive based) [[ScriptInputOutputSchema]].
    +   */
    +  protected def withScriptIOSchema(
    +      inRowFormat: RowFormatContext,
    +      outRowFormat: RowFormatContext,
    +      outRecordReader: Token): ScriptInputOutputSchema = null
    +
    +  /**
    +   * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
    +   * separated) relations here, these get converted into a single plan by condition-less inner join.
    +   */
    +  override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
    +    ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None))
    +  }
    +
    +  /**
    +   * Connect two queries by a Set operator.
    +   *
    +   * Supported Set operators are:
    +   * - UNION [DISTINCT]
    +   * - UNION ALL
    +   * - EXCEPT [DISTINCT]
    +   * - INTERSECT [DISTINCT]
    +   */
    +  override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
    +    val left = plan(ctx.left)
    +    val right = plan(ctx.right)
    +    val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.UNION if all =>
    +        Union(left, right)
    +      case SqlBaseParser.UNION =>
    +        Distinct(Union(left, right))
    +      case SqlBaseParser.INTERSECT if all =>
    +        notSupported("INTERSECT ALL is not supported.", ctx)
    +      case SqlBaseParser.INTERSECT =>
    +        Intersect(left, right)
    +      case SqlBaseParser.EXCEPT if all =>
    +        notSupported("EXCEPT ALL is not supported.", ctx)
    +      case SqlBaseParser.EXCEPT =>
    +        Except(left, right)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[WithWindowDefinition]] operator to a logical plan.
    +   */
    +  private def withWindows(
    +      ctx: WindowsContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Collect all window specifications defined in the WINDOW clause.
    +    val baseWindowMap = ctx.namedWindow.asScala.map {
    +      wCtx =>
    +        (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
    +    }.toMap
    +
    +    // Handle cases like
    +    // window w1 as (partition by p_mfgr order by p_name
    +    //               range between 2 preceding and 2 following),
    +    //        w2 as w1
    +    val windowMap = baseWindowMap.mapValues {
    +      case WindowSpecReference(name) => baseWindowMap(name).asInstanceOf[WindowSpecDefinition]
    +      case spec: WindowSpecDefinition => spec
    +    }
    +    WithWindowDefinition(windowMap, query)
    +  }
    +
    +  /**
    +   * Add an [[Aggregate]] to a logical plan.
    +   */
    +  private def withAggregation(
    +      ctx: AggregationContext,
    +      selectExpressions: Seq[NamedExpression],
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    val groupByExpressions = expressionList(groupingExpressions)
    +
    +    if (GROUPING != null) {
    +      // GROUP BY .... GROUPING SETS (...)
    +      // TODO use new expression set here?
    +      val expressionMap = groupByExpressions.zipWithIndex.toMap
    +      val numExpressions = expressionMap.size
    +      val mask = (1 << numExpressions) - 1
    +      val masks = ctx.groupingSet.asScala.map {
    +        _.expression.asScala.foldLeft(mask) {
    +          case (bitmap, eCtx) =>
    +            // Find the index of the expression.
    +            val e = typedVisit[Expression](eCtx)
    +            val index = expressionMap.find(_._1.semanticEquals(e)).map(_._2).getOrElse(
    +              throw new AnalysisException(
    +                s"${e.treeString} doesn't show up in the GROUP BY list"))
    +            // 0 means that the column at the given index is a grouping column, 1 means it is not,
    +            // so we unset the bit in bitmap.
    +            bitmap & ~(1 << (numExpressions - 1 - index))
    +        }
    +      }
    +      GroupingSets(masks, groupByExpressions, query, selectExpressions)
    +    } else {
    +      // GROUP BY .... (WITH CUBE | WITH ROLLUP)?
    +      val mappedGroupByExpressions = if (CUBE != null) {
    +        Seq(Cube(groupByExpressions))
    +      } else if (ROLLUP != null) {
    +        Seq(Rollup(groupByExpressions))
    +      } else {
    +        groupByExpressions
    +      }
    +      Aggregate(mappedGroupByExpressions, selectExpressions, query)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[Generate]] (Lateral View) to a logical plan.
    +   */
    +  private def withGenerate(
    +      query: LogicalPlan,
    +      ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
    +    val expressions = expressionList(ctx.expression)
    +
    +    // Create the generator.
    +    // TODO Add support for other generators.
    +    val generator = ctx.qualifiedName.getText.toLowerCase match {
    +      case "explode" if expressions.size == 1 =>
    +        Explode(expressions.head)
    +      case "json_tuple" =>
    +        JsonTuple(expressions)
    +      case other =>
    +        notSupported(s"Generator function '$other' is not supported", ctx)
    +    }
    +
    +    Generate(
    +      generator,
    +      join = true,
    +      outer = ctx.OUTER != null,
    +      Some(ctx.tblName.getText.toLowerCase),
    +      ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply),
    +      query)
    +  }
    +
    +  /**
    +   * Create a join between two logical plans.
    +   */
    +  override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) {
    +    val baseJoinType = ctx.joinType match {
    +      case jt if jt.FULL != null => FullOuter
    +      case jt if jt.SEMI != null => LeftSemi
    +      case jt if jt.LEFT != null => LeftOuter
    +      case jt if jt.RIGHT != null => RightOuter
    +      case _ => Inner
    +    }
    +    val joinType = if (ctx.NATURAL != null) {
    +      NaturalJoin(baseJoinType)
    +    } else {
    +      baseJoinType
    +    }
    +
    +    val left = plan(ctx.left)
    +    val right = if (ctx.right != null) {
    +      plan(ctx.right)
    +    } else {
    +      plan(ctx.rightRelation)
    +    }
    +    assert(left != null, "Left side should not be null", ctx)
    +    assert(right != null, "Right side should not be null", ctx)
    +    Join(left, right, joinType, Option(ctx.booleanExpression).map(expression))
    +  }
    +
    +  /**
    +   * Create a sampled relation. This returns a [[Sample]] operator when sampling is requested.
    +   *
    +   * This currently supports the following sampling methods:
    +   * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
    +   * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
    +   * are defined as a number between 0 and 100.
    +   * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
    +   */
    +  override def visitSampledRelation(ctx: SampledRelationContext): LogicalPlan = withOrigin(ctx) {
    +    val relation = plan(ctx.relationPrimary)
    +
    +    // Create a sampled plan if we need one.
    +    def sample(fraction: Double): Sample = {
    +      Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, relation)(true)
    +    }
    +
    +    // Sample the relation if we have to.
    +    relation.optional(ctx.sampleType) {
    +      ctx.sampleType.getType match {
    +        case SqlBaseParser.ROWS =>
    +          Limit(expression(ctx.expression), relation)
    +
    +        case SqlBaseParser.PERCENTLIT =>
    +          val fraction = ctx.percentage.getText.toDouble
    +          // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
    +          // function takes X PERCENT as the input and the range of X is [0, 100], we need to
    +          // adjust the fraction.
    +          val eps = RandomSampler.roundingEpsilon
    +          require(fraction >= 0.0 - eps && fraction <= 100.0 + eps,
    +            s"Sampling fraction ($fraction) must be on interval [0, 100]")
    +          sample(fraction / 100.0d)
    +
    +        case SqlBaseParser.BUCKET if ctx.ON != null =>
    +          notSupported("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx)
    +
    +        case SqlBaseParser.BUCKET =>
    +          sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a sub-query.
    +   */
    +  override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryNoWith)
    +  }
    +
    +  /**
    +   * Create an un-aliased table reference. This is typically used for top-level table references,
    +   * for example:
    +   * {{{
    +   *   INSERT INTO db.tbl2
    +   *   TABLE db.tbl1
    +   * }}}
    +   */
    +  override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None)
    +  }
    +
    +  /**
    +   * Create an aliased table reference. This is typically used in FROM clauses.
    +   */
    +  override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(
    +      visitTableIdentifier(ctx.tableIdentifier),
    +      Option(ctx.identifier).map(_.getText))
    +  }
    +
    +  /**
    +   * Create an inline table (a virtual table in Hive parlance).
    +   */
    +  override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
    +    // Get the backing expressions.
    +    val expressions = ctx.expression.asScala.map(expression)
    +
    +    // Validate and evaluate the rows.
    +    val (structType, structConstructor) = expressions.head.dataType match {
    +      case st: StructType =>
    +        (st, (e: Expression) => e)
    +      case dt =>
    +        val st = CreateStruct(Seq(expressions.head)).dataType
    +        (st, (e: Expression) => CreateStruct(Seq(e)))
    +    }
    +    val rows = expressions.map {
    +      case expression =>
    +        assert(expression.foldable, "All expressions in an inline table must be constants.", ctx)
    +        val safe = Cast(structConstructor(expression), structType)
    +        safe.eval().asInstanceOf[InternalRow]
    +    }
    +
    +    // Construct attributes.
    +    val baseAttributes = structType.toAttributes
    +    val attributes = if (ctx.columnAliases != null) {
    +      val aliases = visitColumnAliases(ctx.columnAliases)
    +      assert(aliases.size == baseAttributes.size,
    +        "Number of aliases must match the number of fields in an inline table.", ctx)
    +      baseAttributes.zip(aliases).map(p => p._1.withName(p._2))
    +    } else {
    +      baseAttributes
    +    }
    +
    +    LocalRelation(attributes, rows)
    +  }
    +
    +  /**
    +   * Create an alias (SubqueryAlias) for a join relation. This is practically the same as
    +   * visitAliasedQuery and visitNamedExpression, ANTLR4 however requires us to use 3 different
    +   * hooks.
    +   */
    +  override def visitAliasedRelation(ctx: AliasedRelationContext): LogicalPlan = withOrigin(ctx) {
    +    aliasPlan(ctx.identifier, plan(ctx.relation()))
    +  }
    +
    +  /**
    +   * Create an alias (SubqueryAlias) for a sub-query. This is practically the same as
    +   * visitAliasedRelation and visitNamedExpression, ANTLR4 however requires us to use 3 different
    +   * hooks.
    +   */
    +  override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) {
    +    aliasPlan(ctx.identifier, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create an alias (SubqueryAlias) for a LogicalPlan. The alias is allowed to be optional.
    +   */
    +  private def aliasPlan(alias: IdentifierContext, plan: LogicalPlan): LogicalPlan = {
    +    plan.optional(alias) {
    +      SubqueryAlias(alias.getText, plan)
    +    }
    +  }
    +
    +  /**
    +   * Create a Sequence of Strings for a parenthesis enclosed alias list.
    +   */
    +  override def visitColumnAliases(ctx: ColumnAliasesContext): Seq[String] = withOrigin(ctx) {
    +    visitColumnAliasList(ctx.columnAliasList)
    +  }
    +
    +  /**
    +   * Create a Sequence of Strings for an alias list.
    +   */
    +  override def visitColumnAliasList(ctx: ColumnAliasListContext): Seq[String] = withOrigin(ctx) {
    +    ctx.identifier.asScala.map(_.getText)
    +  }
    +
    +  /**
    +   * Create a [[TableIdentifier]] from a 'tableName' or 'databaseName'.'tableName' pattern.
    +   */
    +  override def visitTableIdentifier(
    +      ctx: TableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText))
    +  }
    +
    +  /* ********************************************************************************************
    +   * Expression parsing
    +   * ******************************************************************************************** */
    +  private def expression(tree: ParserRuleContext): Expression = typedVisit(tree)
    +
    +  private def expressionList(trees: java.util.List[ExpressionContext]): Seq[Expression] = {
    +    trees.asScala.map(expression)
    +  }
    +
    +  private def invertIfNotDefined(expression: Expression, not: TerminalNode): Expression = {
    +    if (not != null) {
    +      Not(expression)
    +    } else {
    +      expression
    +    }
    +  }
    +
    +  override def visitStar(ctx: StarContext): Expression = withOrigin(ctx) {
    +    UnresolvedStar(Option(ctx.qualifiedName()).map(_.identifier.asScala.map(_.getText)))
    +  }
    +
    +  override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) {
    +    val e = expression(ctx.expression)
    +    if (ctx.identifier != null) {
    +      Alias(e, ctx.identifier.getText)()
    +    } else if (ctx.columnAliases != null) {
    +      MultiAlias(e, visitColumnAliases(ctx.columnAliases))
    +    } else {
    +      e
    +    }
    +  }
    +
    +  override def visitLogicalBinary(ctx: LogicalBinaryContext): Expression = withOrigin(ctx) {
    +    val left = expression(ctx.left)
    +    val right = expression(ctx.right)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.AND =>
    +        And(left, right)
    +      case SqlBaseParser.OR =>
    +        Or(left, right)
    +    }
    +  }
    +
    +  override def visitLogicalNot(ctx: LogicalNotContext): Expression = withOrigin(ctx) {
    +    Not(expression(ctx.booleanExpression()))
    +  }
    +
    +  override def visitExists(ctx: ExistsContext): Expression = {
    +    notSupported("Exists is not supported.", ctx)
    +  }
    +
    +  override def visitComparison(ctx: ComparisonContext): Expression = withOrigin(ctx) {
    +    val left = expression(ctx.value)
    +    val right = expression(ctx.right)
    +    val operator = ctx.comparisonOperator().getChild(0).asInstanceOf[TerminalNode]
    +    operator.getSymbol.getType match {
    +      case SqlBaseParser.EQ =>
    +        EqualTo(left, right)
    +      case SqlBaseParser.NSEQ =>
    +        EqualNullSafe(left, right)
    +      case SqlBaseParser.NEQ =>
    +        Not(EqualTo(left, right))
    +      case SqlBaseParser.LT =>
    +        LessThan(left, right)
    +      case SqlBaseParser.LTE =>
    +        LessThanOrEqual(left, right)
    +      case SqlBaseParser.GT =>
    +        GreaterThan(left, right)
    +      case SqlBaseParser.GTE =>
    +        GreaterThanOrEqual(left, right)
    +    }
    +  }
    +
    +  override def visitBetween(ctx: BetweenContext): Expression = withOrigin(ctx) {
    +    val value = expression(ctx.value)
    +    val between = And(
    +      GreaterThanOrEqual(value, expression(ctx.lower)),
    +      LessThanOrEqual(value, expression(ctx.upper)))
    +    invertIfNotDefined(between, ctx.NOT)
    +  }
    +
    +  override def visitInList(ctx: InListContext): Expression = withOrigin(ctx) {
    +    val in = In(expression(ctx.value), ctx.expression().asScala.map(expression))
    +    invertIfNotDefined(in, ctx.NOT)
    +  }
    +
    +  override def visitInSubquery(ctx: InSubqueryContext): Expression = {
    +    notSupported("IN with a Sub-query is currently not supported.", ctx)
    +  }
    +
    +  override def visitLike(ctx: LikeContext): Expression = {
    +    val left = expression(ctx.value)
    +    val right = expression(ctx.pattern)
    +    val like = ctx.like.getType match {
    +      case SqlBaseParser.LIKE =>
    +        Like(left, right)
    +      case SqlBaseParser.RLIKE =>
    +        RLike(left, right)
    +    }
    +    invertIfNotDefined(like, ctx.NOT)
    +  }
    +
    +  override def visitNullPredicate(ctx: NullPredicateContext): Expression = withOrigin(ctx) {
    +    val value = expression(ctx.value)
    +    if (ctx.NOT != null) {
    +      IsNotNull(value)
    +    } else {
    +      IsNull(value)
    +    }
    +  }
    +
    +  override def visitArithmeticBinary(ctx: ArithmeticBinaryContext): Expression = withOrigin(ctx) {
    +    val left = expression(ctx.left)
    +    val right = expression(ctx.right)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.ASTERISK =>
    +        Multiply(left, right)
    +      case SqlBaseParser.SLASH =>
    +        Divide(left, right)
    +      case SqlBaseParser.PERCENT =>
    +        Remainder(left, right)
    +      case SqlBaseParser.DIV =>
    +        Cast(Divide(left, right), LongType)
    +      case SqlBaseParser.PLUS =>
    +        Add(left, right)
    +      case SqlBaseParser.MINUS =>
    +        Subtract(left, right)
    +      case SqlBaseParser.AMPERSAND =>
    +        BitwiseAnd(left, right)
    +      case SqlBaseParser.HAT =>
    +        BitwiseXor(left, right)
    +      case SqlBaseParser.PIPE =>
    +        BitwiseXor(left, right)
    +    }
    +  }
    +
    +  override def visitArithmeticUnary(ctx: ArithmeticUnaryContext): Expression = withOrigin(ctx) {
    +    val value = expression(ctx.valueExpression)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.PLUS =>
    +        value
    +      case SqlBaseParser.MINUS =>
    +        UnaryMinus(value)
    +      case SqlBaseParser.TILDE =>
    +        BitwiseNot(value)
    +    }
    +  }
    +
    +  override def visitCast(ctx: CastContext): Expression = withOrigin(ctx) {
    +    Cast(expression(ctx.expression), typedVisit(ctx.dataType))
    +  }
    +
    +  override def visitPrimitiveDatatype(ctx: PrimitiveDatatypeContext): DataType = withOrigin(ctx) {
    --- End diff --
    
    I want to replace the current DataType parsing methods with these two methods. This is for a follow-up PR.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57278832
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    +    assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b)))
    +    assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b))
    +  }
    +
    +  test("transform query spec") {
    +    val p = ScriptTransformation(Seq('a, 'b), "func", Seq.empty, table("e"), null)
    +    assertEqual("select transform(a, b) using 'func' from e where f < 10",
    +      p.copy(child = p.child.where('f < 10)))
    +    assertEqual("map(a, b) using 'func' as c, d from e",
    +      p.copy(output = Seq('c.string, 'd.string)))
    +    assertEqual("reduce(a, b) using 'func' as (c: int, d decimal(10, 0)) from e",
    +      p.copy(output = Seq('c.int, 'd.decimal(10, 0))))
    +  }
    +
    +  test("multi select query") {
    +    assertEqual(
    +      "from a select * select * where s < 10",
    +      table("a").select(star()).unionAll(table("a").where('s < 10).select(star())))
    +    intercept(
    +      "from a select * select * from x where a.s < 10",
    +      "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements")
    +    assertEqual(
    +      "from a insert into tbl1 select * insert into tbl2 select * where s < 10",
    +      table("a").select(star()).insertInto("tbl1").unionAll(
    +        table("a").where('s < 10).select(star()).insertInto("tbl2")))
    +  }
    +
    +  test("query organization") {
    +    // Test all valid combinations of order by/sort by/distribute by/cluster by/limit/windows
    +    val baseSql = "select * from t"
    +    val basePlan = table("t").select(star())
    +
    +    val ws = Map("w1" -> WindowSpecDefinition(Seq.empty, Seq.empty, UnspecifiedFrame))
    +    val limitWindowClauses = Seq(
    +      ("", (p: LogicalPlan) => p),
    +      (" limit 10", (p: LogicalPlan) => p.limit(10)),
    +      (" window w1 as ()", (p: LogicalPlan) => WithWindowDefinition(ws, p)),
    +      (" window w1 as () limit 10", (p: LogicalPlan) => WithWindowDefinition(ws, p).limit(10))
    +    )
    +
    +    val orderSortDistrClusterClauses = Seq(
    +      ("", basePlan),
    +      (" order by a, b desc", basePlan.orderBy('a.asc, 'b.desc)),
    +      (" sort by a, b desc", basePlan.sortBy('a.asc, 'b.desc)),
    +      (" distribute by a, b", basePlan.distribute('a, 'b)),
    +      (" distribute by a sort by b", basePlan.distribute('a).sortBy('b.asc)),
    +      (" cluster by a, b", basePlan.distribute('a, 'b).sortBy('a.asc, 'b.asc))
    +    )
    +
    +    orderSortDistrClusterClauses.foreach {
    +      case (s1, p1) =>
    +        limitWindowClauses.foreach {
    +          case (s2, pf2) =>
    +            assertEqual(baseSql + s1 + s2, pf2(p1))
    +        }
    +    }
    +
    +    val msg = "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported"
    +    intercept(s"$baseSql order by a sort by a", msg)
    +    intercept(s"$baseSql cluster by a distribute by a", msg)
    +    intercept(s"$baseSql order by a cluster by a", msg)
    +    intercept(s"$baseSql order by a distribute by a", msg)
    +  }
    +
    +  test("insert into") {
    +    val sql = "select * from t"
    +    val plan = table("t").select(star())
    +    def insert(
    +        partition: Map[String, Option[String]],
    +        overwrite: Boolean = false,
    +        ifNotExists: Boolean = false): LogicalPlan =
    +      InsertIntoTable(table("s"), partition, plan, overwrite, ifNotExists)
    +
    +    // Single inserts
    +    assertEqual(s"insert overwrite table s $sql",
    +      insert(Map.empty, overwrite = true))
    +    assertEqual(s"insert overwrite table s if not exists $sql",
    +      insert(Map.empty, overwrite = true, ifNotExists = true))
    +    assertEqual(s"insert into s $sql",
    +      insert(Map.empty))
    +    assertEqual(s"insert into table s partition (c = 'd', e = 1) $sql",
    +      insert(Map("c" -> Option("d"), "e" -> Option("1"))))
    +    assertEqual(s"insert overwrite table s partition (c = 'd', x) if not exists $sql",
    +      insert(Map("c" -> Option("d"), "x" -> None), overwrite = true, ifNotExists = true))
    +
    +    // Multi insert
    +    val plan2 = table("t").where('x > 5).select(star())
    +    assertEqual("from t insert into s select * limit 1 insert into u select * where x > 5",
    +      InsertIntoTable(
    +        table("s"), Map.empty, plan.limit(1), overwrite = false, ifNotExists = false).unionAll(
    +        InsertIntoTable(
    +          table("u"), Map.empty, plan2, overwrite = false, ifNotExists = false)))
    +  }
    +
    +  test("aggregation") {
    +    val sql = "select a, b, sum(c) as c from d group by a, b"
    +
    +    // Normal
    +    assertEqual(sql, table("d").groupBy('a, 'b)('a, 'b, 'sum.function('c).as("c")))
    +
    +    // Cube
    +    assertEqual(s"$sql with cube",
    +      table("d").groupBy(Cube(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
    +
    +    // Rollup
    +    assertEqual(s"$sql with rollup",
    +      table("d").groupBy(Rollup(Seq('a, 'b)))('a, 'b, 'sum.function('c).as("c")))
    +
    +    // Grouping Sets
    +    assertEqual(s"$sql grouping sets((a, b), (a), ())",
    +      GroupingSets(Seq(0, 1, 3), Seq('a, 'b), table("d"), Seq('a, 'b, 'sum.function('c).as("c"))))
    +    intercept(s"$sql grouping sets((a, b), (c), ())",
    +      "c doesn't show up in the GROUP BY list")
    +  }
    +
    +  test("limit") {
    +    val sql = "select * from t"
    +    val plan = table("t").select(star())
    +    assertEqual(s"$sql limit 10", plan.limit(10))
    +    assertEqual(s"$sql limit cast(9 / 4 as int)", plan.limit(Cast(Literal(9) / 4, IntegerType)))
    +  }
    +
    +  test("window spec") {
    +    // Note that WindowSpecs are testing in the ExpressionParserSuite
    +    val sql = "select * from t"
    +    val plan = table("t").select(star())
    +    val spec = WindowSpecDefinition(Seq('a, 'b), Seq('c.asc),
    +      SpecifiedWindowFrame(RowFrame, ValuePreceding(1), ValueFollowing(1)))
    +
    +    // Test window resolution.
    +    val ws1 = Map("w1" -> spec, "w2" -> spec, "w3" -> spec)
    +    assertEqual(
    +      s"""$sql
    +         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
    +         |       w2 as w1,
    +         |       w3 as w1""".stripMargin,
    +      WithWindowDefinition(ws1, plan))
    +
    +    // Fail with no reference.
    +    intercept(s"$sql window w2 as w1", "Cannot resolve window reference 'w1'")
    +
    +    // Fail when resolved reference is not a window spec.
    +    intercept(
    +      s"""$sql
    +         |window w1 as (partition by a, b order by c rows between 1 preceding and 1 following),
    +         |       w2 as w1,
    +         |       w3 as w2""".stripMargin,
    +      "Window reference 'w2' is not a window specification"
    +    )
    +  }
    +
    +  test("lateral view") {
    +    // Single lateral view
    +    assertEqual(
    +      "select * from t lateral view explode(x) expl as x",
    +      table("t")
    +        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
    +        .select(star()))
    +
    +    // Multiple lateral views
    +    assertEqual(
    +      """select *
    +        |from t
    +        |lateral view explode(x) expl as x
    +        |lateral view outer json_tuple(x, y) jtup q, z""".stripMargin,
    +      table("t")
    +        .generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
    +        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = true, Some("jtup"), Seq("q", "z"))
    +        .select(star()))
    +
    +    // Multi-Insert lateral views.
    +    val from = table("t1").generate(Explode('x), join = true, outer = false, Some("expl"), Seq("x"))
    +    assertEqual(
    +      """from t1
    +        |lateral view explode(x) expl as x
    +        |insert into t2
    +        |select *
    +        |lateral view json_tuple(x, y) jtup q, z
    +        |insert into t3
    +        |select *
    +        |where s < 10
    +      """.stripMargin,
    +      Union(from
    +        .generate(JsonTuple(Seq('x, 'y)), join = true, outer = false, Some("jtup"), Seq("q", "z"))
    +        .select(star())
    +        .insertInto("t2"),
    +        from.where('s < 10).select(star()).insertInto("t3")))
    +
    +    // Unsupported generator.
    +    intercept(
    +      "select * from t lateral view posexplode(x) posexpl as x, y",
    +      "Generator function 'posexplode' is not supported")
    +  }
    +
    +  test("joins") {
    +    val testUnconditionalJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t as tt $sql u",
    +        table("t").as("tt").join(table("u"), jt, None).select(star()))
    +    }
    +    val testConditionalJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t $sql u as uu on a = b",
    +        table("t").join(table("u").as("uu"), jt, Option('a === 'b)).select(star()))
    +    }
    +    val testNaturalJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t tt natural $sql u as uu",
    +        table("t").as("tt").join(table("u").as("uu"), NaturalJoin(jt), None).select(star()))
    +    }
    +    val testUsingJoin = (sql: String, jt: JoinType) => {
    +      assertEqual(
    +        s"select * from t $sql u using(a, b)",
    +        table("t").join(table("u"), UsingJoin(jt, Seq('a.attr, 'b.attr)), None).select(star()))
    +    }
    +    val testAll = Seq(testUnconditionalJoin, testConditionalJoin, testNaturalJoin, testUsingJoin)
    +
    +    def test(sql: String, jt: JoinType, tests: Seq[(String, JoinType) => Unit]): Unit = {
    +      tests.foreach(_(sql, jt))
    +    }
    +    test("cross join", Inner, Seq(testUnconditionalJoin))
    +    test(",", Inner, Seq(testUnconditionalJoin))
    +    test("join", Inner, testAll)
    +    test("inner join", Inner, testAll)
    +    test("left join", LeftOuter, testAll)
    +    test("left outer join", LeftOuter, testAll)
    +    test("right join", RightOuter, testAll)
    +    test("right outer join", RightOuter, testAll)
    +    test("full join", FullOuter, testAll)
    +    test("full outer join", FullOuter, testAll)
    +  }
    +
    +  test("sampled relations") {
    +    val sql = "select * from t"
    +    assertEqual(s"$sql tablesample(100 rows)",
    +      table("t").limit(100).select(star()))
    +    assertEqual(s"$sql as x tablesample(43 percent)",
    +      Sample(0, .43d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
    +    assertEqual(s"$sql as x tablesample(bucket 4 out of 10)",
    +      Sample(0, .4d, withReplacement = false, 10L, table("t").as("x"))(true).select(star()))
    +    intercept(s"$sql as x tablesample(bucket 4 out of 10 on x)",
    +      "TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported")
    +    intercept(s"$sql as x tablesample(bucket 11 out of 10)",
    +      s"Sampling fraction (${11.0/10.0}) must be on interval [0, 1]")
    +  }
    +
    +  test("sub-query") {
    +    val plan = table("t0").select('id)
    +    assertEqual("select id from (t0)", plan)
    +    assertEqual("select id from ((((((t0))))))", plan)
    +    assertEqual(
    +      "(select * from t1) union distinct (select * from t2)",
    +      Distinct(table("t1").select(star()).unionAll(table("t2").select(star()))))
    +    assertEqual(
    +      "select * from ((select * from t1) union (select * from t2)) t",
    +      Distinct(
    +        table("t1").select(star()).unionAll(table("t2").select(star()))).as("t").select(star()))
    +    assertEqual(
    +      """select  id
    +        |from (((select id from t0)
    +        |       union all
    +        |       (select  id from t0))
    +        |      union all
    +        |      (select id from t0)) as u_1
    +      """.stripMargin,
    +      plan.unionAll(plan).unionAll(plan).as("u_1").select('id))
    +  }
    +
    +  test("scalar sub-query") {
    +    assertEqual(
    +      "select (select max(b) from s) ss from t",
    +      table("t").select(ScalarSubquery(table("s").select('max.function('b))).as("ss")))
    +    assertEqual(
    +      "select * from t where a = (select b from s)",
    +      table("t").where('a === ScalarSubquery(table("s").select('b))).select(star()))
    +    assertEqual(
    +      "select g from t group by g having a > (select b from s)",
    +      table("t").groupBy('g)('g).where('a > ScalarSubquery(table("s").select('b))))
    +  }
    +
    +  test("table reference") {
    +    assertEqual("table t", table("t"))
    +    assertEqual("table d.t", table("d", "t"))
    +  }
    +
    +  test("inline table") {
    +    assertEqual("values 1, 2, 3, 4", LocalRelation.fromExternalRows(
    --- End diff --
    
    This PostgreSQL doc page might be helpful: http://www.postgresql.org/docs/9.5/static/sql-values.html


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-199785459
  
    **[Test build #53764 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53764/consoleFull)** for PR 11557 at commit [`a5d12ba`](https://github.com/apache/spark/commit/a5d12ba93a3c29194800d5e7de11704a96741833).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57148298
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/CatalystQlSuite.scala ---
    @@ -21,15 +21,18 @@ import org.apache.spark.sql.AnalysisException
     import org.apache.spark.sql.catalyst.TableIdentifier
     import org.apache.spark.sql.catalyst.analysis._
     import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical._
     import org.apache.spark.unsafe.types.CalendarInterval
     
     class CatalystQlSuite extends PlanTest {
    -  val parser = new CatalystQl()
    +  val parser = CatalystSqlParser
    --- End diff --
    
    Since we still keep the antlr 3 code, I think we should create a new `CatalystSqlParserSuite` and copy code here to it. We can remove this file after we complete migrating to antlr 4.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193374259
  
    **[Test build #52572 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52572/consoleFull)** for PR 11557 at commit [`f95840d`](https://github.com/apache/spark/commit/f95840d00bcc2ba4324e9760907515e436b15edc).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198146148
  
    **[Test build #53488 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/53488/consoleFull)** for PR 11557 at commit [`c5d0bcf`](https://github.com/apache/spark/commit/c5d0bcff0d41c1f302d842d6e111fa85a8baf74b).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57165747
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala ---
    @@ -161,6 +161,10 @@ package object dsl {
         def lower(e: Expression): Expression = Lower(e)
         def sqrt(e: Expression): Expression = Sqrt(e)
         def abs(e: Expression): Expression = Abs(e)
    +    def all(names: String*): Expression = names match {
    --- End diff --
    
    +1 for `star`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-200368863
  
    @srowen thanks for the heads-up. I'll add an entry to `LICENSE`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-199512015
  
    btw here's another idea -- we don't need to do it as part of this pr.
    
    it'd still be great to have those parser tests compare plans. The problem in the past was it's annoying to update those plans when the parser changes. However, if we can make these compare against some reference files that are just the plans in their json representations, it becomes much easier to update these plans.
    
    Same thing goes for optimization rules.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57271242
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/ExpressionParserSuite.scala ---
    @@ -0,0 +1,494 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _}
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.PlanTest
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +
    +/**
    + * Test basic expression parsing. If a type of expression is supported it should be tested here.
    + *
    + * Please note that some of the expressions test don't have to be sound expressions, only their
    + * structure needs to be valid. Unsound expressions should be caught by the Analyzer or
    + * CheckAnalysis classes.
    + */
    +class ExpressionParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, e: Expression): Unit = {
    +    compareExpressions(parseExpression(sqlCommand), e)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parseExpression(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("star expressions") {
    +    // Global Star
    +    assertEqual("*", UnresolvedStar(None))
    +
    +    // Targeted Star
    +    assertEqual("a.b.*", UnresolvedStar(Option(Seq("a", "b"))))
    +  }
    +
    +  // NamedExpression (Alias/Multialias)
    +  test("named expressions") {
    +    // No Alias
    +    val r0 = 'a
    +    assertEqual("a", r0)
    +
    +    // Single Alias.
    +    val r1 = 'a as "b"
    +    assertEqual("a as b", r1)
    +    assertEqual("a b", r1)
    +
    +    // Multi-Alias
    +    assertEqual("a as (b, c)", MultiAlias('a, Seq("b", "c")))
    +    assertEqual("a() (b, c)", MultiAlias('a.function(), Seq("b", "c")))
    +
    +    // Numeric literals without a space between the literal qualifier and the alias, should not be
    +    // interpreted as such. An unresolved reference should be returned instead.
    +    // TODO add the JIRA-ticket number.
    +    assertEqual("1SL", Symbol("1SL"))
    +
    +    // Aliased star is allowed.
    +    assertEqual("a.* b", UnresolvedStar(Option(Seq("a"))) as 'b)
    +  }
    +
    +  test("binary logical expressions") {
    +    // And
    +    assertEqual("a and b", 'a && 'b)
    +
    +    // Or
    +    assertEqual("a or b", 'a || 'b)
    +
    +    // Combination And/Or check precedence
    +    assertEqual("a and b or c and d", ('a && 'b) || ('c && 'd))
    +    assertEqual("a or b or c and d", 'a || 'b || ('c && 'd))
    +
    +    // Multiple AND/OR get converted into a balanced tree
    +    assertEqual("a or b or c or d or e or f", (('a || 'b) || 'c) || (('d || 'e) || 'f))
    +    assertEqual("a and b and c and d and e and f", (('a && 'b) && 'c) && (('d && 'e) && 'f))
    +  }
    +
    +  test("long binary logical expressions") {
    +    def testVeryBinaryExpression(op: String, clazz: Class[_]): Unit = {
    +      val sql = (1 to 1000).map(x => s"$x == $x").mkString(op)
    +      val e = parseExpression(sql)
    +      assert(e.collect { case _: EqualTo => true }.size === 1000)
    +      assert(e.collect { case x if clazz.isInstance(x) => true }.size === 999)
    +    }
    +    testVeryBinaryExpression(" AND ", classOf[And])
    +    testVeryBinaryExpression(" OR ", classOf[Or])
    +  }
    +
    +  test("not expressions") {
    +    assertEqual("not a", !'a)
    +    assertEqual("!a", !'a)
    +    assertEqual("not true > true", Not(GreaterThan(true, true)))
    +  }
    +
    +  test("exists expression") {
    +    intercept("exists (select 1 from b where b.x = a.x)", "EXISTS clauses are not supported")
    +  }
    +
    +  test("comparison expressions") {
    +    assertEqual("a = b", 'a === 'b)
    +    assertEqual("a == b", 'a === 'b)
    +    assertEqual("a <=> b", 'a <=> 'b)
    +    assertEqual("a <> b", 'a =!= 'b)
    +    assertEqual("a != b", 'a =!= 'b)
    +    assertEqual("a < b", 'a < 'b)
    +    assertEqual("a <= b", 'a <= 'b)
    +    assertEqual("a > b", 'a > 'b)
    +    assertEqual("a >= b", 'a >= 'b)
    +  }
    +
    +  test("between expressions") {
    +    assertEqual("a between b and c", 'a >= 'b && 'a <= 'c)
    +    assertEqual("a not between b and c", !('a >= 'b && 'a <= 'c))
    +  }
    +
    +  test("in expressions") {
    +    assertEqual("a in (b, c, d)", 'a in ('b, 'c, 'd))
    +    assertEqual("a not in (b, c, d)", !('a in ('b, 'c, 'd)))
    +  }
    +
    +  test("in sub-query") {
    +    intercept("a in (select b from c)", "IN with a Sub-query is currently not supported")
    +  }
    +
    +  test("like expressions") {
    +    assertEqual("a like 'pattern%'", 'a like "pattern%")
    +    assertEqual("a not like 'pattern%'", !('a like "pattern%"))
    +    assertEqual("a rlike 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not rlike 'pattern%'", !('a rlike "pattern%"))
    +    assertEqual("a regexp 'pattern%'", 'a rlike "pattern%")
    +    assertEqual("a not regexp 'pattern%'", !('a rlike "pattern%"))
    +  }
    +
    +  test("is null expressions") {
    +    assertEqual("a is null", 'a.isNull)
    +    assertEqual("a is not null", 'a.isNotNull)
    +  }
    +
    +  test("binary arithmetic expressions") {
    +    // Simple operations
    +    assertEqual("a * b", 'a * 'b)
    +    assertEqual("a / b", 'a / 'b)
    +    assertEqual("a DIV b", ('a / 'b).cast(LongType))
    +    assertEqual("a % b", 'a % 'b)
    +    assertEqual("a + b", 'a + 'b)
    +    assertEqual("a - b", 'a - 'b)
    +    assertEqual("a & b", 'a & 'b)
    +    assertEqual("a ^ b", 'a ^ 'b)
    +    assertEqual("a | b", 'a | 'b)
    +
    +    // Check precedences
    +    assertEqual(
    +      "a * t | b ^ c & d - e + f % g DIV h / i * k",
    +      'a * 't | ('b ^ ('c & ('d - 'e + (('f % 'g / 'h).cast(LongType) / 'i * 'k)))))
    +  }
    +
    +  test("unary arithmetic expressions") {
    +    assertEqual("+a", 'a)
    +    assertEqual("-a", -'a)
    +    assertEqual("~a", ~'a)
    +    assertEqual("-+~~a", -(~(~'a)))
    +  }
    +
    +  test("cast expressions") {
    +    // Note that DataType parsing is tested elsewhere.
    +    assertEqual("cast(a as int)", 'a.cast(IntegerType))
    +    assertEqual("cast(a as timestamp)", 'a.cast(TimestampType))
    +    assertEqual("cast(a as array<int>)", 'a.cast(ArrayType(IntegerType)))
    +    assertEqual("cast(cast(a as int) as long)", 'a.cast(IntegerType).cast(LongType))
    +  }
    +
    +  test("function expressions") {
    +    assertEqual("foo()", 'foo.function())
    +    assertEqual("foo.bar()", Symbol("foo.bar").function())
    --- End diff --
    
    when will we have a prefix for function?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-195808208
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53011/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57226227
  
    --- Diff: LICENSE ---
    @@ -238,6 +238,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt.
          (BSD 3 Clause) netlib core (com.github.fommil.netlib:core:1.1.2 - https://github.com/fommil/netlib-java/core)
          (BSD 3 Clause) JPMML-Model (org.jpmml:pmml-model:1.2.7 - https://github.com/jpmml/jpmml-model)
          (BSD License) AntLR Parser Generator (antlr:antlr:2.7.7 - http://www.antlr.org/)
    +     (BSD License) ANTLR 4.5.2-1 (org.antlr:antlr4:4.5.2-1 - http://wwww.antlr.org/)
    --- End diff --
    
    @srowen is this sufficient?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-200562102
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202495324
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/54320/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193513755
  
    **[Test build #52613 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52613/consoleFull)** for PR 11557 at commit [`723edfb`](https://github.com/apache/spark/commit/723edfba11c40e832916d90b5d1453c926317022).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by yhuai <gi...@git.apache.org>.
Github user yhuai commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-202562672
  
    https://github.com/apache/spark/pull/12010


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by liancheng <gi...@git.apache.org>.
Github user liancheng commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r57279004
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ng/PlanParserSuite.scala ---
    @@ -0,0 +1,408 @@
    +/*
    + * 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.parser.ng
    +
    +import org.apache.spark.sql.Row
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types.IntegerType
    +
    +class PlanParserSuite extends PlanTest {
    +  import CatalystSqlParser._
    +  import org.apache.spark.sql.catalyst.dsl.expressions._
    +  import org.apache.spark.sql.catalyst.dsl.plans._
    +
    +  def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
    +    comparePlans(parsePlan(sqlCommand), plan)
    +  }
    +
    +  def intercept(sqlCommand: String, messages: String*): Unit = {
    +    val e = intercept[ParseException](parsePlan(sqlCommand))
    +    messages.foreach { message =>
    +      assert(e.message.contains(message))
    +    }
    +  }
    +
    +  test("case insensitive") {
    +    val plan = table("a").select(star())
    +    assertEqual("sELEct * FroM a", plan)
    +    assertEqual("select * fRoM a", plan)
    +    assertEqual("SELECT * FROM a", plan)
    +  }
    +
    +  test("show functions") {
    +    assertEqual("show functions", ShowFunctions(None, None))
    +    assertEqual("show functions foo", ShowFunctions(None, Some("foo")))
    +    assertEqual("show functions foo.bar", ShowFunctions(Some("foo"), Some("bar")))
    +    assertEqual("show functions 'foo\\\\.*'", ShowFunctions(None, Some("foo\\.*")))
    +    intercept("show functions foo.bar.baz", "SHOW FUNCTIONS unsupported name")
    +  }
    +
    +  test("describe function") {
    +    assertEqual("describe function bar", DescribeFunction("bar", isExtended = false))
    +    assertEqual("describe function extended bar", DescribeFunction("bar", isExtended = true))
    +    assertEqual("describe function foo.bar", DescribeFunction("foo.bar", isExtended = false))
    +    assertEqual("describe function extended f.bar", DescribeFunction("f.bar", isExtended = true))
    +  }
    +
    +  test("set operations") {
    +    val a = table("a").select(star())
    +    val b = table("b").select(star())
    +
    +    assertEqual("select * from a union select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union distinct select * from b", Distinct(a.unionAll(b)))
    +    assertEqual("select * from a union all select * from b", a.unionAll(b))
    +    assertEqual("select * from a except select * from b", a.except(b))
    +    intercept("select * from a except all select * from b", "EXCEPT ALL is not supported.")
    +    assertEqual("select * from a except distinct select * from b", a.except(b))
    +    assertEqual("select * from a intersect select * from b", a.intersect(b))
    +    intercept("select * from a intersect all select * from b", "INTERSECT ALL is not supported.")
    +    assertEqual("select * from a intersect distinct select * from b", a.intersect(b))
    +  }
    +
    +  test("common table expressions") {
    +    def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = {
    +      val ctes = namedPlans.map {
    +        case (name, cte) =>
    +          name -> SubqueryAlias(name, cte)
    +      }.toMap
    +      With(plan, ctes)
    +    }
    +    assertEqual(
    +      "with cte1 as (select * from a) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> table("a").select(star())))
    +    assertEqual(
    +      "with cte1 (select 1) select * from cte1",
    +      cte(table("cte1").select(star()), "cte1" -> OneRowRelation.select(1)))
    +    assertEqual(
    +      "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2",
    +      cte(table("cte2").select(star()),
    +        "cte1" -> OneRowRelation.select(1),
    +        "cte2" -> table("cte1").select(star())))
    +    intercept(
    +      "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1",
    +      "Name 'cte1' is used for multiple common table expressions")
    +  }
    +
    +  test("simple select query") {
    +    assertEqual("select 1", OneRowRelation.select(1))
    +    assertEqual("select a, b", OneRowRelation.select('a, 'b))
    +    assertEqual("select a, b from db.c", table("db", "c").select('a, 'b))
    +    assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b))
    +    assertEqual("select a, b from db.c having x < 1", table("db", "c").select('a, 'b).where('x < 1))
    --- End diff --
    
    Agree. There should be at least one aggregate function being `SELECT`-ed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198468855
  
    **[Test build #2651 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2651/consoleFull)** for PR 11557 at commit [`c5d0bcf`](https://github.com/apache/spark/commit/c5d0bcff0d41c1f302d842d6e111fa85a8baf74b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193704911
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198136162
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53480/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #11557: [SPARK-13713][SQL] Migrate parser from ANTLR3 to ...

Posted by hvanhovell <gi...@git.apache.org>.
Github user hvanhovell commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11557#discussion_r73653787
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ng/AstBuilder.scala ---
    @@ -0,0 +1,1452 @@
    +/*
    + * 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.parser.ng
    +
    +import java.sql.{Date, Timestamp}
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.antlr.v4.runtime.{ParserRuleContext, Token}
    +import org.antlr.v4.runtime.tree.{ParseTree, TerminalNode}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
    +import org.apache.spark.sql.catalyst.analysis._
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.parser.ng.SqlBaseParser._
    +import org.apache.spark.sql.catalyst.plans._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.types._
    +import org.apache.spark.unsafe.types.CalendarInterval
    +import org.apache.spark.util.random.RandomSampler
    +
    +/**
    + * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
    + * TableIdentifier.
    + */
    +class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
    +  import ParserUtils._
    +
    +  protected def typedVisit[T](ctx: ParseTree): T = {
    +    ctx.accept(this).asInstanceOf[T]
    +  }
    +
    +  override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) {
    +    visit(ctx.statement).asInstanceOf[LogicalPlan]
    +  }
    +
    +  override def visitSingleExpression(ctx: SingleExpressionContext): Expression = withOrigin(ctx) {
    +    visitNamedExpression(ctx.namedExpression)
    +  }
    +
    +  override def visitSingleTableIdentifier(
    +      ctx: SingleTableIdentifierContext): TableIdentifier = withOrigin(ctx) {
    +    visitTableIdentifier(ctx.tableIdentifier)
    +  }
    +
    +  override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
    +    visit(ctx.dataType).asInstanceOf[DataType]
    +  }
    +
    +  /* ********************************************************************************************
    +   * Plan parsing
    +   * ******************************************************************************************** */
    +  protected def plan(tree: ParserRuleContext): LogicalPlan = typedVisit(tree)
    +
    +  /**
    +   * Make sure we do not try to create a plan for a native command.
    +   */
    +  override def visitExecuteNativeCommand(ctx: ExecuteNativeCommandContext): LogicalPlan = null
    +
    +  /**
    +   * Create a plan for a SHOW FUNCTIONS command.
    +   */
    +  override def visitShowFunctions(ctx: ShowFunctionsContext): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    if (qualifiedName != null) {
    +      val names = qualifiedName().identifier().asScala.map(_.getText).toList
    +      names match {
    +        case db :: name :: Nil =>
    +          ShowFunctions(Some(db), Some(name))
    +        case name :: Nil =>
    +          ShowFunctions(None, Some(name))
    +        case _ =>
    +          throw new ParseException("SHOW FUNCTIONS unsupported name", ctx)
    +      }
    +    } else if (pattern != null) {
    +      ShowFunctions(None, Some(string(pattern)))
    +    } else {
    +      ShowFunctions(None, None)
    +    }
    +  }
    +
    +  /**
    +   * Create a plan for a DESCRIBE FUNCTION command.
    +   */
    +  override def visitDescribeFunction(ctx: DescribeFunctionContext): LogicalPlan = withOrigin(ctx) {
    +    val functionName = ctx.qualifiedName().identifier().asScala.map(_.getText).mkString(".")
    +    DescribeFunction(functionName, ctx.EXTENDED != null)
    +  }
    +
    +  /**
    +   * Create a top-level plan with Common Table Expressions.
    +   */
    +  override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) {
    +    val query = plan(ctx.queryNoWith)
    +
    +    // Apply CTEs
    +    query.optional(ctx.ctes) {
    +      val ctes = ctx.ctes.namedQuery.asScala.map {
    +        case nCtx =>
    +          val namedQuery = visitNamedQuery(nCtx)
    +          (namedQuery.alias, namedQuery)
    +      }
    +
    +      // Check for duplicate names.
    +      ctes.groupBy(_._1).filter(_._2.size > 1).foreach {
    +        case (name, _) =>
    +          throw new ParseException(
    +            s"Name '$name' is used for multiple common table expressions", ctx)
    +      }
    +
    +      With(query, ctes.toMap)
    +    }
    +  }
    +
    +  /**
    +   * Create a named logical plan.
    +   *
    +   * This is only used for Common Table Expressions.
    +   */
    +  override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) {
    +    SubqueryAlias(ctx.name.getText, plan(ctx.queryNoWith))
    +  }
    +
    +  /**
    +   * Create a logical plan which allows for multiple inserts using one 'from' statement. These
    +   * queries have the following SQL form:
    +   * {{{
    +   *   [WITH cte...]?
    +   *   FROM src
    +   *   [INSERT INTO tbl1 SELECT *]+
    +   * }}}
    +   * For example:
    +   * {{{
    +   *   FROM db.tbl1 A
    +   *   INSERT INTO dbo.tbl1 SELECT * WHERE A.value = 10 LIMIT 5
    +   *   INSERT INTO dbo.tbl2 SELECT * WHERE A.value = 12
    +   * }}}
    +   * This (Hive) feature cannot be combined with set-operators.
    +   */
    +  override def visitMultiInsertQuery(ctx: MultiInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    val from = visitFromClause(ctx.fromClause)
    +
    +    // Build the insert clauses.
    +    val inserts = ctx.multiInsertQueryBody.asScala.map {
    +      body =>
    +        assert(body.querySpecification.fromClause == null,
    +          "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements",
    +          body)
    +
    +        withQuerySpecification(body.querySpecification, from).
    +          // Add organization statements.
    +          optionalMap(body.queryOrganization)(withQueryResultClauses).
    +          // Add insert.
    +          optionalMap(body.insertInto())(withInsertInto)
    +    }
    +
    +    // If there are multiple INSERTS just UNION them together into one query.
    +    inserts match {
    +      case Seq(query) => query
    +      case queries => Union(queries)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a regular (single-insert) query.
    +   */
    +  override def visitSingleInsertQuery(
    +      ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryTerm).
    +      // Add organization statements.
    +      optionalMap(ctx.queryOrganization)(withQueryResultClauses).
    +      // Add insert.
    +      optionalMap(ctx.insertInto())(withInsertInto)
    +  }
    +
    +  /**
    +   * Add an INSERT INTO [TABLE]/INSERT OVERWRITE TABLE operation to the logical plan.
    +   */
    +  private def withInsertInto(
    +      ctx: InsertIntoContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    val tableIdent = visitTableIdentifier(ctx.tableIdentifier)
    +    val partitionKeys = Option(ctx.partitionSpec).map(visitPartitionSpec).getOrElse(Map.empty)
    +
    +    InsertIntoTable(
    +      UnresolvedRelation(tableIdent, None),
    +      partitionKeys,
    +      query,
    +      ctx.OVERWRITE != null,
    +      ctx.EXISTS != null)
    +  }
    +
    +  /**
    +   * Create a partition specification map.
    +   */
    +  override def visitPartitionSpec(
    +      ctx: PartitionSpecContext): Map[String, Option[String]] = withOrigin(ctx) {
    +    ctx.partitionVal.asScala.map { pVal =>
    +      val name = pVal.identifier.getText.toLowerCase
    +      val value = Option(pVal.constant).map(visitStringConstant)
    +      name -> value
    +    }.toMap
    +  }
    +
    +  /**
    +   * Create a partition specification map without optional values.
    +   */
    +  protected def visitNonOptionalPartitionSpec(
    +      ctx: PartitionSpecContext): Map[String, String] = withOrigin(ctx) {
    +    visitPartitionSpec(ctx).mapValues(_.orNull).map(identity)
    +  }
    +
    +  /**
    +   * Convert a constant of any type into a string. This is typically used in DDL commands, and its
    +   * main purpose is to prevent slight differences due to back to back conversions i.e.:
    +   * String -> Literal -> String.
    +   */
    +  protected def visitStringConstant(ctx: ConstantContext): String = withOrigin(ctx) {
    +    ctx match {
    +      case s: StringLiteralContext => createString(s)
    +      case o => o.getText
    +    }
    +  }
    +
    +  /**
    +   * Add ORDER BY/SORT BY/CLUSTER BY/DISTRIBUTE BY/LIMIT/WINDOWS clauses to the logical plan. These
    +   * clauses determine the shape (ordering/partitioning/rows) of the query result.
    +   */
    +  private def withQueryResultClauses(
    +      ctx: QueryOrganizationContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause.
    +    val withOrder = if (
    +      !order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // ORDER BY ...
    +      Sort(order.asScala.map(visitSortItem), global = true, query)
    +    } else if (order.isEmpty && !sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ...
    +      Sort(sort.asScala.map(visitSortItem), global = false, query)
    +    } else if (order.isEmpty && sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // DISTRIBUTE BY ...
    +      RepartitionByExpression(expressionList(distributeBy), query)
    +    } else if (order.isEmpty && !sort.isEmpty && !distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // SORT BY ... DISTRIBUTE BY ...
    +      Sort(
    +        sort.asScala.map(visitSortItem),
    +        global = false,
    +        RepartitionByExpression(expressionList(distributeBy), query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && !clusterBy.isEmpty) {
    +      // CLUSTER BY ...
    +      val expressions = expressionList(clusterBy)
    +      Sort(
    +        expressions.map(SortOrder(_, Ascending)),
    +        global = false,
    +        RepartitionByExpression(expressions, query))
    +    } else if (order.isEmpty && sort.isEmpty && distributeBy.isEmpty && clusterBy.isEmpty) {
    +      // [EMPTY]
    +      query
    +    } else {
    +      throw new ParseException(
    +        "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", ctx)
    +    }
    +
    +    // WINDOWS
    +    val withWindow = withOrder.optionalMap(windows)(withWindows)
    +
    +    // LIMIT
    +    withWindow.optional(limit) {
    +      Limit(typedVisit(limit), withWindow)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan using a query specification.
    +   */
    +  override def visitQuerySpecification(
    +      ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) {
    +    val from = OneRowRelation.optional(ctx.fromClause) {
    +      visitFromClause(ctx.fromClause)
    +    }
    +    withQuerySpecification(ctx, from)
    +  }
    +
    +  /**
    +   * Add a query specification to a logical plan. The query specification is the core of the logical
    +   * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE),
    +   * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place.
    +   *
    +   * Note that query hints are ignored (both by the parser and the builder).
    +   */
    +  private def withQuerySpecification(
    +      ctx: QuerySpecificationContext,
    +      relation: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +
    +    // WHERE
    +    def filter(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = {
    +      Filter(expression(ctx), plan)
    +    }
    +
    +    // Expressions.
    +    val expressions = Option(namedExpressionSeq).toSeq
    +      .flatMap(_.namedExpression.asScala)
    +      .map(typedVisit[Expression])
    +
    +    // Create either a transform or a regular query.
    +    val specType = Option(kind).map(_.getType).getOrElse(SqlBaseParser.SELECT)
    +    specType match {
    +      case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM =>
    +        // Transform
    +
    +        // Add where.
    +        val withFilter = relation.optionalMap(where)(filter)
    +
    +        // Create the attributes.
    +        val (attributes, schemaLess) = if (colTypeList != null) {
    +          // Typed return columns.
    +          (createStructType(colTypeList).toAttributes, false)
    +        } else if (identifierSeq != null) {
    +          // Untyped return columns.
    +          val attrs = visitIdentifierSeq(identifierSeq).map { name =>
    +            AttributeReference(name, StringType, nullable = true)()
    +          }
    +          (attrs, false)
    +        } else {
    +          (Seq(AttributeReference("key", StringType)(),
    +            AttributeReference("value", StringType)()), true)
    +        }
    +
    +        // Create the transform.
    +        ScriptTransformation(
    +          expressions,
    +          string(script),
    +          attributes,
    +          withFilter,
    +          withScriptIOSchema(inRowFormat, recordWriter, outRowFormat, recordReader, schemaLess))
    +
    +      case SqlBaseParser.SELECT =>
    +        // Regular select
    +
    +        // Add lateral views.
    +        val withLateralView = ctx.lateralView.asScala.foldLeft(relation)(withGenerate)
    +
    +        // Add where.
    +        val withFilter = withLateralView.optionalMap(where)(filter)
    +
    +        // Add aggregation or a project.
    +        val namedExpressions = expressions.map {
    +          case e: NamedExpression => e
    +          case e: Expression => UnresolvedAlias(e)
    +        }
    +        val withProject = if (aggregation != null) {
    +          withAggregation(aggregation, namedExpressions, withFilter)
    +        } else if (namedExpressions.nonEmpty) {
    +          Project(namedExpressions, withFilter)
    +        } else {
    +          withFilter
    +        }
    +
    +        // Having
    +        val withHaving = withProject.optional(having) {
    +          // Note that we added a cast to boolean. If the expression itself is already boolean,
    +          // the optimizer will get rid of the unnecessary cast.
    +          Filter(Cast(expression(having), BooleanType), withProject)
    +        }
    +
    +        // Distinct
    +        val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) {
    +          Distinct(withHaving)
    +        } else {
    +          withHaving
    +        }
    +
    +        // Window
    +        withDistinct.optionalMap(windows)(withWindows)
    +    }
    +  }
    +
    +  /**
    +   * Create a (Hive based) [[ScriptInputOutputSchema]].
    +   */
    +  protected def withScriptIOSchema(
    +      inRowFormat: RowFormatContext,
    +      recordWriter: Token,
    +      outRowFormat: RowFormatContext,
    +      recordReader: Token,
    +      schemaLess: Boolean): ScriptInputOutputSchema = null
    +
    +  /**
    +   * Create a logical plan for a given 'FROM' clause. Note that we support multiple (comma
    +   * separated) relations here, these get converted into a single plan by condition-less inner join.
    +   */
    +  override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) {
    +    val from = ctx.relation.asScala.map(plan).reduceLeft(Join(_, _, Inner, None))
    +    ctx.lateralView.asScala.foldLeft(from)(withGenerate)
    +  }
    +
    +  /**
    +   * Connect two queries by a Set operator.
    +   *
    +   * Supported Set operators are:
    +   * - UNION [DISTINCT]
    +   * - UNION ALL
    +   * - EXCEPT [DISTINCT]
    +   * - INTERSECT [DISTINCT]
    +   */
    +  override def visitSetOperation(ctx: SetOperationContext): LogicalPlan = withOrigin(ctx) {
    +    val left = plan(ctx.left)
    +    val right = plan(ctx.right)
    +    val all = Option(ctx.setQuantifier()).exists(_.ALL != null)
    +    ctx.operator.getType match {
    +      case SqlBaseParser.UNION if all =>
    +        Union(left, right)
    +      case SqlBaseParser.UNION =>
    +        Distinct(Union(left, right))
    +      case SqlBaseParser.INTERSECT if all =>
    +        throw new ParseException("INTERSECT ALL is not supported.", ctx)
    +      case SqlBaseParser.INTERSECT =>
    +        Intersect(left, right)
    +      case SqlBaseParser.EXCEPT if all =>
    +        throw new ParseException("EXCEPT ALL is not supported.", ctx)
    +      case SqlBaseParser.EXCEPT =>
    +        Except(left, right)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[WithWindowDefinition]] operator to a logical plan.
    +   */
    +  private def withWindows(
    +      ctx: WindowsContext,
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Collect all window specifications defined in the WINDOW clause.
    +    val baseWindowMap = ctx.namedWindow.asScala.map {
    +      wCtx =>
    +        (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
    +    }.toMap
    +
    +    // Handle cases like
    +    // window w1 as (partition by p_mfgr order by p_name
    +    //               range between 2 preceding and 2 following),
    +    //        w2 as w1
    +    val windowMapView = baseWindowMap.mapValues {
    +      case WindowSpecReference(name) =>
    +        baseWindowMap.get(name) match {
    +          case Some(spec: WindowSpecDefinition) =>
    +            spec
    +          case Some(ref) =>
    +            throw new ParseException(s"Window reference '$name' is not a window specification", ctx)
    +          case None =>
    +            throw new ParseException(s"Cannot resolve window reference '$name'", ctx)
    +        }
    +      case spec: WindowSpecDefinition => spec
    +    }
    +
    +    // Note that mapValues creates a view instead of materialized map. We force materialization by
    +    // mapping over identity.
    +    WithWindowDefinition(windowMapView.map(identity), query)
    +  }
    +
    +  /**
    +   * Add an [[Aggregate]] to a logical plan.
    +   */
    +  private def withAggregation(
    +      ctx: AggregationContext,
    +      selectExpressions: Seq[NamedExpression],
    +      query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    import ctx._
    +    val groupByExpressions = expressionList(groupingExpressions)
    +
    +    if (GROUPING != null) {
    +      // GROUP BY .... GROUPING SETS (...)
    +      val expressionMap = groupByExpressions.zipWithIndex.toMap
    +      val numExpressions = expressionMap.size
    +      val mask = (1 << numExpressions) - 1
    +      val masks = ctx.groupingSet.asScala.map {
    +        _.expression.asScala.foldLeft(mask) {
    +          case (bitmap, eCtx) =>
    +            // Find the index of the expression.
    +            val e = typedVisit[Expression](eCtx)
    +            val index = expressionMap.find(_._1.semanticEquals(e)).map(_._2).getOrElse(
    +              throw new ParseException(
    +                s"$e doesn't show up in the GROUP BY list", ctx))
    +            // 0 means that the column at the given index is a grouping column, 1 means it is not,
    +            // so we unset the bit in bitmap.
    +            bitmap & ~(1 << (numExpressions - 1 - index))
    +        }
    +      }
    +      GroupingSets(masks, groupByExpressions, query, selectExpressions)
    +    } else {
    +      // GROUP BY .... (WITH CUBE | WITH ROLLUP)?
    +      val mappedGroupByExpressions = if (CUBE != null) {
    +        Seq(Cube(groupByExpressions))
    +      } else if (ROLLUP != null) {
    +        Seq(Rollup(groupByExpressions))
    +      } else {
    +        groupByExpressions
    +      }
    +      Aggregate(mappedGroupByExpressions, selectExpressions, query)
    +    }
    +  }
    +
    +  /**
    +   * Add a [[Generate]] (Lateral View) to a logical plan.
    +   */
    +  private def withGenerate(
    +      query: LogicalPlan,
    +      ctx: LateralViewContext): LogicalPlan = withOrigin(ctx) {
    +    val expressions = expressionList(ctx.expression)
    +
    +    // Create the generator.
    +    val generator = ctx.qualifiedName.getText.toLowerCase match {
    +      case "explode" if expressions.size == 1 =>
    +        Explode(expressions.head)
    +      case "json_tuple" =>
    +        JsonTuple(expressions)
    +      case other =>
    +        withGenerator(other, expressions, ctx)
    +    }
    +
    +    Generate(
    +      generator,
    +      join = true,
    +      outer = ctx.OUTER != null,
    +      Some(ctx.tblName.getText.toLowerCase),
    +      ctx.colName.asScala.map(_.getText).map(UnresolvedAttribute.apply),
    +      query)
    +  }
    +
    +  /**
    +   * Create a [[Generator]]. Override this method in order to support custom Generators.
    +   */
    +  protected def withGenerator(
    +      name: String,
    +      expressions: Seq[Expression],
    +      ctx: LateralViewContext): Generator = {
    +    throw new ParseException(s"Generator function '$name' is not supported", ctx)
    +  }
    +
    +  /**
    +   * Create a joins between two or more logical plans.
    +   */
    +  override def visitJoinRelation(ctx: JoinRelationContext): LogicalPlan = withOrigin(ctx) {
    +    /** Build a join between two plans. */
    +    def join(ctx: JoinRelationContext, left: LogicalPlan, right: LogicalPlan): Join = {
    +      val baseJoinType = ctx.joinType match {
    +        case null => Inner
    +        case jt if jt.FULL != null => FullOuter
    +        case jt if jt.SEMI != null => LeftSemi
    +        case jt if jt.LEFT != null => LeftOuter
    +        case jt if jt.RIGHT != null => RightOuter
    +        case _ => Inner
    +      }
    +
    +      // Resolve the join type and join condition
    +      val (joinType, condition) = Option(ctx.joinCriteria) match {
    +        case Some(c) if c.USING != null =>
    +          val columns = c.identifier.asScala.map { column =>
    +            UnresolvedAttribute.quoted(column.getText)
    +          }
    +          (UsingJoin(baseJoinType, columns), None)
    +        case Some(c) if c.booleanExpression != null =>
    +          (baseJoinType, Option(expression(c.booleanExpression)))
    +        case None if ctx.NATURAL != null =>
    +          (NaturalJoin(baseJoinType), None)
    +        case None =>
    +          (baseJoinType, None)
    +      }
    +      Join(left, right, joinType, condition)
    +    }
    +
    +    // Handle all consecutive join clauses. ANTLR produces a right nested tree in which the the
    +    // first join clause is at the top. However fields of previously referenced tables can be used
    +    // in following join clauses. The tree needs to be reversed in order to make this work.
    +    var result = plan(ctx.left)
    +    var current = ctx
    +    while (current != null) {
    +      current.right match {
    +        case right: JoinRelationContext =>
    +          result = join(current, result, plan(right.left))
    +          current = right
    +        case right =>
    +          result = join(current, result, plan(right))
    +          current = null
    +      }
    +    }
    +    result
    +  }
    +
    +  /**
    +   * Add a [[Sample]] to a logical plan.
    +   *
    +   * This currently supports the following sampling methods:
    +   * - TABLESAMPLE(x ROWS): Sample the table down to the given number of rows.
    +   * - TABLESAMPLE(x PERCENT): Sample the table down to the given percentage. Note that percentages
    +   * are defined as a number between 0 and 100.
    +   * - TABLESAMPLE(BUCKET x OUT OF y): Sample the table down to a 'x' divided by 'y' fraction.
    +   */
    +  private def withSample(ctx: SampleContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) {
    +    // Create a sampled plan if we need one.
    +    def sample(fraction: Double): Sample = {
    +      // The range of fraction accepted by Sample is [0, 1]. Because Hive's block sampling
    +      // function takes X PERCENT as the input and the range of X is [0, 100], we need to
    +      // adjust the fraction.
    +      val eps = RandomSampler.roundingEpsilon
    +      assert(fraction >= 0.0 - eps && fraction <= 1.0 + eps,
    +        s"Sampling fraction ($fraction) must be on interval [0, 1]",
    +        ctx)
    +      Sample(0.0, fraction, withReplacement = false, (math.random * 1000).toInt, query)(true)
    +    }
    +
    +    ctx.sampleType.getType match {
    +      case SqlBaseParser.ROWS =>
    +        Limit(expression(ctx.expression), query)
    +
    +      case SqlBaseParser.PERCENTLIT =>
    +        val fraction = ctx.percentage.getText.toDouble
    +        sample(fraction / 100.0d)
    +
    +      case SqlBaseParser.BUCKET if ctx.ON != null =>
    +        throw new ParseException("TABLESAMPLE(BUCKET x OUT OF y ON id) is not supported", ctx)
    +
    +      case SqlBaseParser.BUCKET =>
    +        sample(ctx.numerator.getText.toDouble / ctx.denominator.getText.toDouble)
    +    }
    +  }
    +
    +  /**
    +   * Create a logical plan for a sub-query.
    +   */
    +  override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) {
    +    plan(ctx.queryNoWith)
    +  }
    +
    +  /**
    +   * Create an un-aliased table reference. This is typically used for top-level table references,
    +   * for example:
    +   * {{{
    +   *   INSERT INTO db.tbl2
    +   *   TABLE db.tbl1
    +   * }}}
    +   */
    +  override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) {
    +    UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier), None)
    +  }
    +
    +  /**
    +   * Create an aliased table reference. This is typically used in FROM clauses.
    +   */
    +  override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) {
    +    val table = UnresolvedRelation(
    +      visitTableIdentifier(ctx.tableIdentifier),
    +      Option(ctx.identifier).map(_.getText))
    +    table.optionalMap(ctx.sample)(withSample)
    +  }
    +
    +  /**
    +   * Create an inline table (a virtual table in Hive parlance).
    +   */
    +  override def visitInlineTable(ctx: InlineTableContext): LogicalPlan = withOrigin(ctx) {
    +    // Get the backing expressions.
    +    val expressions = ctx.expression.asScala.map { eCtx =>
    +      val e = expression(eCtx)
    +      assert(e.foldable, "All expressions in an inline table must be constants.", eCtx)
    +      e
    +    }
    +
    +    // Validate and evaluate the rows.
    +    val (structType, structConstructor) = expressions.head.dataType match {
    --- End diff --
    
    We infer the type of the first row, and then cast all other rows to the first row's type. See: https://github.com/apache/spark/pull/11557/files/6f1c535162397f01acf0405bdc80b8c4c141fc64#diff-05222a1d022860b15a2fc1ec3445b368R696


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193183236
  
    **[Test build #52550 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52550/consoleFull)** for PR 11557 at commit [`06303bc`](https://github.com/apache/spark/commit/06303bcc2801b3f7520c3174a982e21056e86386).
     * This patch **fails build dependency tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193554967
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193521618
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-193445754
  
    **[Test build #52594 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52594/consoleFull)** for PR 11557 at commit [`340522e`](https://github.com/apache/spark/commit/340522eeb37e36bc50d41152b4ec274c8ced3095).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-13713][SQL] Migrate parser from ANTLR3 ...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/11557#issuecomment-198167947
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/53488/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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