You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by liancheng <gi...@git.apache.org> on 2014/10/08 01:10:24 UTC

[GitHub] spark pull request: [SPARK-3654][SQL][WIP] Unifies SQL and HiveQL ...

GitHub user liancheng opened a pull request:

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

    [SPARK-3654][SQL][WIP] Unifies SQL and HiveQL parsers

    This PR is a follow up of #2590, and tries to introduce a top level SQL parser entry point for all SQL dialects supported by Spark SQL.
    
    A top level parser `SparkSQLParser` is introduced to handle the syntaxes that all SQL dialects should recognize (e.g. `CACHE TABLE`, `UNCACHE TABLE` and `SET`, etc.). For all the syntaxes this parser doesn't recognize directly, it fallbacks to a specified function that tries to parse arbitrary input to a `LogicalPlan`. This function is typically another parser combinator like `SqlParser`. DDL syntaxes introduced in #2475 can be moved to here.
    
    The `ExtendedHiveQlParser` now only handle Hive specific extensions.

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

    $ git pull https://github.com/liancheng/spark gen-sql-parser

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

    https://github.com/apache/spark/pull/2698.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 #2698
    
----
commit 53ae25288ca0cf79193362c71f596982d38ba45a
Author: Cheng Lian <li...@gmail.com>
Date:   2014-10-07T22:51:57Z

    Unifies SQL and HiveQL parsers

----


---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58281309
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21423/consoleFull) for   PR 2698 at commit [`9ebc290`](https://github.com/apache/spark/commit/9ebc29017519c82a7d4720e77e58900e565a4148).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  protected case class Keyword(str: String)`
      * `class SqlLexical(val keywords: Seq[String]) extends StdLexical `
      * `  case class FloatLit(chars: String) extends Token `
      * `class SqlParser extends AbstractSparkSQLParser `
      * `case class SetCommand(kv: Option[(String, Option[String])]) extends Command `
      * `case class ShellCommand(cmd: String) extends Command`
      * `case class SourceCommand(filePath: String) extends Command`
      * `case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribute])(`



---
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-3654][SQL] Unifies SQL and HiveQL parse...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58572818
  
    Can one of the admins verify this patch?


---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58278318
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21421/consoleFull) for   PR 2698 at commit [`53ae252`](https://github.com/apache/spark/commit/53ae25288ca0cf79193362c71f596982d38ba45a).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  protected case class Keyword(str: String)`
      * `class SqlLexical(val keywords: Seq[String]) extends StdLexical `
      * `  case class FloatLit(chars: String) extends Token `
      * `class SqlParser extends AbstractSparkSQLParser `
      * `case class SetCommand(kv: Option[(String, Option[String])]) extends Command `
      * `case class ShellCommand(cmd: String) extends Command`
      * `case class SourceCommand(filePath: String) extends Command`
      * `case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribute])(`



---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58279665
  
    Scala style checking doesn't like the `digit.+` DSL syntax in the parser, I'm falling back to `rep1`.


---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58359149
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21461/consoleFull) for   PR 2698 at commit [`a4b0fc4`](https://github.com/apache/spark/commit/a4b0fc4fc4e3c0ffe7d251dc3e22e96ad68c44dc).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  protected case class Keyword(str: String)`
      * `class SqlLexical(val keywords: Seq[String]) extends StdLexical `
      * `  case class FloatLit(chars: String) extends Token `
      * `class SqlParser extends AbstractSparkSQLParser `
      * `case class SetCommand(kv: Option[(String, Option[String])]) extends Command `
      * `case class ShellCommand(cmd: String) extends Command`
      * `case class SourceCommand(filePath: String) extends Command`
      * `case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribute])(`



---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58278212
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21421/consoleFull) for   PR 2698 at commit [`53ae252`](https://github.com/apache/spark/commit/53ae25288ca0cf79193362c71f596982d38ba45a).
     * This patch merges cleanly.


---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#discussion_r18556146
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala ---
    @@ -39,9 +39,9 @@ case class NativeCommand(cmd: String) extends Command {
     }
     
     /**
    - * Commands of the form "SET (key) (= value)".
    + * Commands of the form "SET [key [= value] ]".
      */
    -case class SetCommand(key: Option[String], value: Option[String]) extends Command {
    +case class SetCommand(kv: Option[(String, Option[String])]) extends Command {
    --- End diff --
    
    Although looks trickier, `Option[(String, Option[String])]` captures the syntax and semantics of `SET` command better, and simplifies parsing logic.


---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58280230
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21423/consoleFull) for   PR 2698 at commit [`9ebc290`](https://github.com/apache/spark/commit/9ebc29017519c82a7d4720e77e58900e565a4148).
     * This patch merges cleanly.


---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58278323
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21421/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-3654][SQL] Unifies SQL and HiveQL parse...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58461743
  
    @marmbrus Please help review. `ADD JAR` and `ADD FILE` are still Hive specific. I'd like to move them to the top level parser in another 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-3654][SQL] Unifies SQL and HiveQL parse...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58597520
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21559/consoleFull) for   PR 2698 at commit [`ceada76`](https://github.com/apache/spark/commit/ceada769af4d2b9d2df7f0bf9013a3ef7a3cb196).
     * This patch merges cleanly.


---
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-3654][SQL] Unifies SQL and HiveQL parse...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58466598
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21525/consoleFull) for   PR 2698 at commit [`ba2c121`](https://github.com/apache/spark/commit/ba2c1214c20e75853b7541f1744f9d1761d97a5d).
     * This patch merges cleanly.


---
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-3654][SQL] Unifies SQL and HiveQL parse...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58600390
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21559/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-3654][SQL] Unifies SQL and HiveQL parse...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58468683
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21525/consoleFull) for   PR 2698 at commit [`ba2c121`](https://github.com/apache/spark/commit/ba2c1214c20e75853b7541f1744f9d1761d97a5d).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  protected case class Keyword(str: String)`
      * `class SqlLexical(val keywords: Seq[String]) extends StdLexical `
      * `  case class FloatLit(chars: String) extends Token `
      * `class SqlParser extends AbstractSparkSQLParser `
      * `case class SetCommand(kv: Option[(String, Option[String])]) extends Command `
      * `case class ShellCommand(cmd: String) extends Command`
      * `case class SourceCommand(filePath: String) extends Command`
      * `case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribute])(`



---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58282482
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21426/consoleFull) for   PR 2698 at commit [`cc14609`](https://github.com/apache/spark/commit/cc14609f24b1731925cc72bc51847bc08c036087).
     * This patch merges cleanly.


---
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-3654][SQL] Unifies SQL and HiveQL parse...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58467387
  
    This PR conflicts with #2678. It would be good to merge that one first, and then I'll rebase.


---
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-3654][SQL] Unifies SQL and HiveQL parse...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58600386
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21559/consoleFull) for   PR 2698 at commit [`ceada76`](https://github.com/apache/spark/commit/ceada769af4d2b9d2df7f0bf9013a3ef7a3cb196).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  protected case class Keyword(str: String)`
      * `class SqlLexical(val keywords: Seq[String]) extends StdLexical `
      * `  case class FloatLit(chars: String) extends Token `
      * `class SqlParser extends AbstractSparkSQLParser `
      * `case class SetCommand(kv: Option[(String, Option[String])]) extends Command `
      * `case class ShellCommand(cmd: String) extends Command`
      * `case class SourceCommand(filePath: String) extends Command`
      * `case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribute])(`



---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58283616
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21426/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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#discussion_r18556938
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala ---
    @@ -0,0 +1,186 @@
    +/*
    + * 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
    +
    +import scala.language.implicitConversions
    +import scala.util.parsing.combinator.lexical.StdLexical
    +import scala.util.parsing.combinator.syntactical.StandardTokenParsers
    +import scala.util.parsing.combinator.{PackratParsers, RegexParsers}
    +import scala.util.parsing.input.CharArrayReader.EofCh
    +
    +import org.apache.spark.sql.catalyst.plans.logical._
    +
    +private[sql] abstract class AbstractSparkSQLParser
    +  extends StandardTokenParsers with PackratParsers {
    +
    +  def apply(input: String): LogicalPlan = phrase(start)(new lexical.Scanner(input)) match {
    +    case Success(plan, _) => plan
    +    case failureOrError => sys.error(failureOrError.toString)
    +  }
    +
    +  protected case class Keyword(str: String)
    +
    +  protected def start: Parser[LogicalPlan]
    +
    +  // Returns the whole input string
    +  protected lazy val wholeInput: Parser[String] = new Parser[String] {
    +    def apply(in: Input): ParseResult[String] =
    +      Success(in.source.toString, in.drop(in.source.length()))
    +  }
    +
    +  // Returns the rest of the input string that are not parsed yet
    +  protected lazy val restInput: Parser[String] = new Parser[String] {
    +    def apply(in: Input): ParseResult[String] =
    +      Success(in.source.toString, in.drop(in.source.length()))
    +  }
    +}
    +
    +class SqlLexical(val keywords: Seq[String]) extends StdLexical {
    +  case class FloatLit(chars: String) extends Token {
    +    override def toString = chars
    +  }
    +
    +  reserved ++= keywords.flatMap(w => allCaseVersions(w))
    +
    +  delimiters += (
    +    "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")",
    +    ",", ";", "%", "{", "}", ":", "[", "]", "."
    +  )
    +
    +  override lazy val token: Parser[Token] =
    +    ( identChar ~ (identChar | digit).* ^^
    +      { case first ~ rest => processIdent((first :: rest).mkString) }
    +    | rep1(digit) ~ ('.' ~> digit.*).? ^^ {
    +        case i ~ None    => NumericLit(i.mkString)
    +        case i ~ Some(d) => FloatLit(i.mkString + "." + d.mkString)
    +      }
    +    | '\'' ~> chrExcept('\'', '\n', EofCh).* <~ '\'' ^^
    +      { case chars => StringLit(chars mkString "") }
    +    | '"' ~> chrExcept('"', '\n', EofCh).* <~ '"' ^^
    +      { case chars => StringLit(chars mkString "") }
    +    | EofCh ^^^ EOF
    +    | '\'' ~> failure("unclosed string literal")
    +    | '"' ~> failure("unclosed string literal")
    +    | delim
    +    | failure("illegal character")
    +    )
    +
    +  override def identChar = letter | elem('_')
    +
    +  override def whitespace: Parser[Any] =
    +    ( whitespaceChar
    +    | '/' ~ '*' ~ comment
    +    | '/' ~ '/' ~ chrExcept(EofCh, '\n').*
    +    | '#' ~ chrExcept(EofCh, '\n').*
    +    | '-' ~ '-' ~ chrExcept(EofCh, '\n').*
    +    | '/' ~ '*' ~ failure("unclosed comment")
    +    ).*
    +
    +  /** Generate all variations of upper and lower case of a given string */
    +  def allCaseVersions(s: String, prefix: String = ""): Stream[String] = {
    +    if (s == "") {
    +      Stream(prefix)
    +    } else {
    +      allCaseVersions(s.tail, prefix + s.head.toLower) ++
    +        allCaseVersions(s.tail, prefix + s.head.toUpper)
    +    }
    +  }
    +}
    +
    +/**
    + * The top level Spark SQL parser. This parser recognizes syntaxes that are available for all SQL
    + * dialects supported by Spark SQL, and delegates all the other syntaxes to the `fallback` parser.
    + *
    + * @param fallback A function that parses an input string to a logical plan
    + */
    +private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends AbstractSparkSQLParser {
    +
    +  // A parser for the key-value part of the "SET [key = [value ]]" syntax
    +  private object SetCommandParser extends RegexParsers {
    +    private val key: Parser[String] = "(?m)[^=]+".r
    +
    +    private val value: Parser[String] = "(?m).+$".r
    +
    +    private val pair: Parser[LogicalPlan] =
    +      (key ~ ("=".r ~> value).?).? ^^ {
    +        case None => SetCommand(None)
    +        case Some(k ~ v) => SetCommand(Some(k.trim -> v.map(_.trim)))
    +      }
    +
    +    def apply(input: String): LogicalPlan = {
    +      parseAll(pair, input).get
    --- End diff --
    
    Debugging code, forgot to remove 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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58283612
  
      [QA tests have finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21426/consoleFull) for   PR 2698 at commit [`cc14609`](https://github.com/apache/spark/commit/cc14609f24b1731925cc72bc51847bc08c036087).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  protected case class Keyword(str: String)`
      * `class SqlLexical(val keywords: Seq[String]) extends StdLexical `
      * `  case class FloatLit(chars: String) extends Token `
      * `class SqlParser extends AbstractSparkSQLParser `
      * `case class SetCommand(kv: Option[(String, Option[String])]) extends Command `
      * `case class ShellCommand(cmd: String) extends Command`
      * `case class SourceCommand(filePath: String) extends Command`
      * `case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribute])(`



---
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-3654][SQL] Unifies SQL and HiveQL parse...

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

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


---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58281314
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21423/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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58353197
  
      [QA tests have started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/21461/consoleFull) for   PR 2698 at commit [`a4b0fc4`](https://github.com/apache/spark/commit/a4b0fc4fc4e3c0ffe7d251dc3e22e96ad68c44dc).
     * This patch merges cleanly.


---
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-3654][SQL][WIP] Unifies SQL and HiveQL ...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58359162
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21461/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-3654][SQL] Unifies SQL and HiveQL parse...

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

    https://github.com/apache/spark/pull/2698#issuecomment-58468687
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/21525/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