Skip to content

Commit d15d54f

Browse files
committed
Unifies SQL and HiveQL parsers
1 parent 4e9b551 commit d15d54f

File tree

10 files changed

+404
-389
lines changed

10 files changed

+404
-389
lines changed
Lines changed: 186 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,186 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.catalyst
19+
20+
import scala.language.implicitConversions
21+
import scala.util.parsing.combinator.lexical.StdLexical
22+
import scala.util.parsing.combinator.syntactical.StandardTokenParsers
23+
import scala.util.parsing.combinator.{PackratParsers, RegexParsers}
24+
import scala.util.parsing.input.CharArrayReader.EofCh
25+
26+
import org.apache.spark.sql.catalyst.plans.logical._
27+
28+
private[sql] abstract class AbstractSparkSQLParser
29+
extends StandardTokenParsers with PackratParsers {
30+
31+
def apply(input: String): LogicalPlan = phrase(start)(new lexical.Scanner(input)) match {
32+
case Success(plan, _) => plan
33+
case failureOrError => sys.error(failureOrError.toString)
34+
}
35+
36+
protected case class Keyword(str: String)
37+
38+
protected def start: Parser[LogicalPlan]
39+
40+
// Returns the whole input string
41+
protected lazy val wholeInput: Parser[String] = new Parser[String] {
42+
def apply(in: Input): ParseResult[String] =
43+
Success(in.source.toString, in.drop(in.source.length()))
44+
}
45+
46+
// Returns the rest of the input string that are not parsed yet
47+
protected lazy val restInput: Parser[String] = new Parser[String] {
48+
def apply(in: Input): ParseResult[String] =
49+
Success(in.source.toString, in.drop(in.source.length()))
50+
}
51+
}
52+
53+
class SqlLexical(val keywords: Seq[String]) extends StdLexical {
54+
case class FloatLit(chars: String) extends Token {
55+
override def toString = chars
56+
}
57+
58+
reserved ++= keywords.flatMap(w => allCaseVersions(w))
59+
60+
delimiters += (
61+
"@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")",
62+
",", ";", "%", "{", "}", ":", "[", "]", "."
63+
)
64+
65+
override lazy val token: Parser[Token] =
66+
( identChar ~ (identChar | digit).* ^^
67+
{ case first ~ rest => processIdent((first :: rest).mkString) }
68+
| digit.+ ~ ('.' ~> digit.*).? ^^ {
69+
case i ~ None => NumericLit(i.mkString)
70+
case i ~ Some(d) => FloatLit(i.mkString + "." + d.mkString)
71+
}
72+
| '\'' ~> chrExcept('\'', '\n', EofCh).* <~ '\'' ^^
73+
{ case chars => StringLit(chars mkString "") }
74+
| '"' ~> chrExcept('"', '\n', EofCh).* <~ '"' ^^
75+
{ case chars => StringLit(chars mkString "") }
76+
| EofCh ^^^ EOF
77+
| '\'' ~> failure("unclosed string literal")
78+
| '"' ~> failure("unclosed string literal")
79+
| delim
80+
| failure("illegal character")
81+
)
82+
83+
override def identChar = letter | elem('_')
84+
85+
override def whitespace: Parser[Any] =
86+
( whitespaceChar
87+
| '/' ~ '*' ~ comment
88+
| '/' ~ '/' ~ chrExcept(EofCh, '\n').*
89+
| '#' ~ chrExcept(EofCh, '\n').*
90+
| '-' ~ '-' ~ chrExcept(EofCh, '\n').*
91+
| '/' ~ '*' ~ failure("unclosed comment")
92+
).*
93+
94+
/** Generate all variations of upper and lower case of a given string */
95+
def allCaseVersions(s: String, prefix: String = ""): Stream[String] = {
96+
if (s == "") {
97+
Stream(prefix)
98+
} else {
99+
allCaseVersions(s.tail, prefix + s.head.toLower) ++
100+
allCaseVersions(s.tail, prefix + s.head.toUpper)
101+
}
102+
}
103+
}
104+
105+
/**
106+
* The top level Spark SQL parser. This parser recognizes syntaxes that are available for all SQL
107+
* dialects supported by Spark SQL, and delegates all the other syntaxes to the `fallback` parser.
108+
*
109+
* @param fallback A function that parses an input string to a logical plan
110+
*/
111+
private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends AbstractSparkSQLParser {
112+
113+
// A parser for the key-value part of the "SET [key = [value ]]" syntax
114+
private object SetCommandParser extends RegexParsers {
115+
private val key: Parser[String] = "(?m)[^=]+".r
116+
117+
private val value: Parser[String] = "(?m).+$".r
118+
119+
private val pair: Parser[LogicalPlan] =
120+
(key ~ ("=".r ~> value).?).? ^^ {
121+
case None => SetCommand(None)
122+
case Some(k ~ v) => SetCommand(Some(k.trim -> v.map(_.trim)))
123+
}
124+
125+
def apply(input: String): LogicalPlan = {
126+
parseAll(pair, input).get
127+
parseAll(pair, input) match {
128+
case Success(plan, _) => plan
129+
case x => sys.error(x.toString)
130+
}
131+
}
132+
}
133+
134+
private val AS = Keyword("AS")
135+
private val CACHE = Keyword("CACHE")
136+
private val LAZY = Keyword("LAZY")
137+
private val SET = Keyword("SET")
138+
private val TABLE = Keyword("TABLE")
139+
private val SOURCE = Keyword("SOURCE")
140+
private val UNCACHE = Keyword("UNCACHE")
141+
142+
protected implicit def asParser(k: Keyword): Parser[String] =
143+
lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _)
144+
145+
private val reservedWords =
146+
this.getClass
147+
.getMethods
148+
.filter(_.getReturnType == classOf[Keyword])
149+
.map(_.invoke(this).asInstanceOf[Keyword].str)
150+
151+
override val lexical = new SqlLexical(reservedWords)
152+
153+
override protected lazy val start: Parser[LogicalPlan] =
154+
cache | uncache | set | shell | source | others
155+
156+
private lazy val cache: Parser[LogicalPlan] =
157+
CACHE ~> LAZY.? ~ (TABLE ~> ident) ~ (AS ~> restInput).? ^^ {
158+
case isLazy ~ tableName ~ plan =>
159+
CacheTableCommand(tableName, plan.map(fallback), isLazy.isDefined)
160+
}
161+
162+
private lazy val uncache: Parser[LogicalPlan] =
163+
UNCACHE ~ TABLE ~> ident ^^ {
164+
case tableName => UncacheTableCommand(tableName)
165+
}
166+
167+
private lazy val set: Parser[LogicalPlan] =
168+
SET ~> restInput ^^ {
169+
case input => SetCommandParser(input)
170+
}
171+
172+
private lazy val shell: Parser[LogicalPlan] =
173+
"!" ~> restInput ^^ {
174+
case input => ShellCommand(input)
175+
}
176+
177+
private lazy val source: Parser[LogicalPlan] =
178+
SOURCE ~> restInput ^^ {
179+
case input => SourceCommand(input)
180+
}
181+
182+
private lazy val others: Parser[LogicalPlan] =
183+
wholeInput ^^ {
184+
case input => fallback(input)
185+
}
186+
}

0 commit comments

Comments
 (0)