Skip to content

Commit 838f701

Browse files
committed
Remove DDLParser.
1 parent d800c58 commit 838f701

File tree

7 files changed

+27
-201
lines changed

7 files changed

+27
-201
lines changed

project/MimaExcludes.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -151,6 +151,8 @@ object MimaExcludes {
151151
ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log_="),
152152
ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log_"),
153153
ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log__=")
154+
) ++ Seq(
155+
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.execution.datasources.DDLParser")
154156
)
155157
case v if v.startsWith("1.6") =>
156158
Seq(

sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/ExpressionParser.g

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -493,6 +493,15 @@ descFuncNames
493493
| functionIdentifier
494494
;
495495
496+
looseIdentifier
497+
:
498+
Identifier
499+
| looseNonReserved -> Identifier[$looseNonReserved.text]
500+
// If it decides to support SQL11 reserved keywords, i.e., useSQL11ReservedKeywordsForIdentifier()=false,
501+
// the sql11keywords in existing q tests will NOT be added back.
502+
| {useSQL11ReservedKeywordsForIdentifier()}? sql11ReservedKeywordsUsedAsIdentifier -> Identifier[$sql11ReservedKeywordsUsedAsIdentifier.text]
503+
;
504+
496505
identifier
497506
:
498507
Identifier
@@ -518,6 +527,10 @@ principalIdentifier
518527
| QuotedIdentifier
519528
;
520529
530+
looseNonReserved
531+
: nonReserved | KW_FROM | KW_TO
532+
;
533+
521534
//The new version of nonReserved + sql11ReservedKeywordsUsedAsIdentifier = old version of nonReserved
522535
//Non reserved keywords are basically the keywords that can be used as identifiers.
523536
//All the KW_* are automatically not only keywords, but also reserved keywords.

sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SparkSqlParser.g

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -935,6 +935,7 @@ createTableStatement
935935
columnNameTypeList?
936936
$p
937937
tableOpts?
938+
selectStatementWithCTE?
938939
)
939940
->
940941
^(TOK_CREATETABLE $name $temp? $ext? ifNotExists?
@@ -1808,8 +1809,8 @@ optionKeyValue
18081809
@init { pushMsg("table's option specification", state); }
18091810
@after { popMsg(state); }
18101811
:
1811-
Identifier StringLiteral
1812-
-> ^(TOK_TABLEOPTION Identifier StringLiteral)
1812+
looseIdentifier StringLiteral
1813+
-> ^(TOK_TABLEOPTION looseIdentifier StringLiteral)
18131814
;
18141815

18151816
tableOpts

sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala

Lines changed: 1 addition & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -205,9 +205,6 @@ class SQLContext private[sql](
205205
@transient
206206
protected[sql] lazy val optimizer: Optimizer = new SparkOptimizer(this)
207207

208-
@transient
209-
protected[sql] val ddlParser = new DDLParser(sqlParser)
210-
211208
@transient
212209
protected[sql] val sqlParser = new SparkSQLParser(getSQLDialect())
213210

@@ -231,7 +228,7 @@ class SQLContext private[sql](
231228
}
232229
}
233230

234-
protected[sql] def parseSql(sql: String): LogicalPlan = ddlParser.parse(sql, false)
231+
protected[sql] def parseSql(sql: String): LogicalPlan = sqlParser.parsePlan(sql)
235232

236233
protected[sql] def executeSql(sql: String):
237234
org.apache.spark.sql.execution.QueryExecution = executePlan(parseSql(sql))

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -98,6 +98,11 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly
9898
}
9999

100100
if (asClause.isDefined) {
101+
if (columns.isDefined) {
102+
throw new DDLException(
103+
"a CREATE TABLE AS SELECT statement does not allow column definitions.")
104+
}
105+
101106
val mode = if (allowExisting.isDefined) {
102107
SaveMode.Ignore
103108
} else if (temp.isDefined) {

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala

Lines changed: 0 additions & 192 deletions
This file was deleted.

sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with
105105
sql("SELECT a, b FROM jsonTable"),
106106
sql("SELECT a, b FROM jt").collect())
107107

108-
val message = intercept[DDLException]{
108+
val message = intercept[AnalysisException]{
109109
sql(
110110
s"""
111111
|CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable
@@ -156,7 +156,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with
156156
}
157157

158158
test("CREATE TEMPORARY TABLE AS SELECT with IF NOT EXISTS is not allowed") {
159-
val message = intercept[DDLException]{
159+
val message = intercept[AnalysisException]{
160160
sql(
161161
s"""
162162
|CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable
@@ -173,7 +173,7 @@ class CreateTableAsSelectSuite extends DataSourceTest with SharedSQLContext with
173173
}
174174

175175
test("a CTAS statement with column definitions is not allowed") {
176-
intercept[DDLException]{
176+
intercept[AnalysisException]{
177177
sql(
178178
s"""
179179
|CREATE TEMPORARY TABLE jsonTable (a int, b string)

0 commit comments

Comments
 (0)