Skip to content

Commit

Permalink
[SPARK-27890][SQL] Improve SQL parser error message for character-onl…
Browse files Browse the repository at this point in the history
…y identifier with hyphens except those in expressions

## What changes were proposed in this pull request?

Current SQL parser's error message for hyphen-connected identifiers without surrounding backquotes(e.g. hyphen-table) is confusing for end users. A possible approach to tackle this is to explicitly capture these wrong usages in the SQL parser. In this way, the end users can fix these errors more quickly.

For example, for a simple query such as `SELECT * FROM test-table`, the original error message is
```
Error in SQL statement: ParseException:
mismatched input '-' expecting <EOF>(line 1, pos 18)
```
which can be confusing in a large query.

After the fix, the error message is:
```
Error in query:
Possibly unquoted identifier test-table detected. Please consider quoting it with back-quotes as `test-table`(line 1, pos 14)

== SQL ==
SELECT * FROM test-table
--------------^^^
```
which is easier for end users to identify the issue and fix.

We safely augmented the current grammar rule to explicitly capture these error cases. The error handling logic is implemented in the SQL parsing listener `PostProcessor`.

However, note that for cases such as `a - my-func(b)`, the parser can't actually tell whether this should be ``a -`my-func`(b) `` or `a - my - func(b)`. Therefore for these cases, we leave the parser as is. Also, in this patch we only provide better error messages for character-only identifiers.

## How was this patch tested?
Adding new unit tests.

Closes #24749 from yeshengm/hyphen-ident.

Authored-by: Yesheng Ma <kimi.ysma@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
  • Loading branch information
yeshengm authored and gatorsmile committed Jun 19, 2019
1 parent 15de6d0 commit 7b7f16f
Show file tree
Hide file tree
Showing 5 changed files with 169 additions and 35 deletions.
Expand Up @@ -82,13 +82,15 @@ singleTableSchema
statement
: query #statementDefault
| ctes? dmlStatementNoWith #dmlStatement
| USE db=identifier #use
| CREATE database (IF NOT EXISTS)? identifier
| USE db=errorCapturingIdentifier #use
| CREATE database (IF NOT EXISTS)? db=errorCapturingIdentifier
((COMMENT comment=STRING) |
locationSpec |
(WITH DBPROPERTIES tablePropertyList))* #createDatabase
| ALTER database identifier SET DBPROPERTIES tablePropertyList #setDatabaseProperties
| DROP database (IF EXISTS)? identifier (RESTRICT | CASCADE)? #dropDatabase
| ALTER database db=errorCapturingIdentifier
SET DBPROPERTIES tablePropertyList #setDatabaseProperties
| DROP database (IF EXISTS)? db=errorCapturingIdentifier
(RESTRICT | CASCADE)? #dropDatabase
| SHOW DATABASES (LIKE? pattern=STRING)? #showDatabases
| createTableHeader ('(' colTypeList ')')? tableProvider
((OPTIONS options=tablePropertyList) |
Expand Down Expand Up @@ -135,7 +137,8 @@ statement
(ALTER | CHANGE) COLUMN? qualifiedName
(TYPE dataType)? (COMMENT comment=STRING)? colPosition? #alterTableColumn
| ALTER TABLE tableIdentifier partitionSpec?
CHANGE COLUMN? identifier colType colPosition? #changeColumn
CHANGE COLUMN?
colName=errorCapturingIdentifier colType colPosition? #changeColumn
| ALTER TABLE tableIdentifier (partitionSpec)?
SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe
| ALTER TABLE tableIdentifier (partitionSpec)?
Expand Down Expand Up @@ -172,20 +175,20 @@ statement
| DROP TEMPORARY? FUNCTION (IF EXISTS)? qualifiedName #dropFunction
| EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN | COST)?
statement #explain
| SHOW TABLES ((FROM | IN) db=identifier)?
| SHOW TABLES ((FROM | IN) db=errorCapturingIdentifier)?
(LIKE? pattern=STRING)? #showTables
| SHOW TABLE EXTENDED ((FROM | IN) db=identifier)?
| SHOW TABLE EXTENDED ((FROM | IN) db=errorCapturingIdentifier)?
LIKE pattern=STRING partitionSpec? #showTable
| SHOW TBLPROPERTIES table=tableIdentifier
('(' key=tablePropertyKey ')')? #showTblProperties
| SHOW COLUMNS (FROM | IN) tableIdentifier
((FROM | IN) db=identifier)? #showColumns
((FROM | IN) db=errorCapturingIdentifier)? #showColumns
| SHOW PARTITIONS tableIdentifier partitionSpec? #showPartitions
| SHOW identifier? FUNCTIONS
(LIKE? (qualifiedName | pattern=STRING))? #showFunctions
| SHOW CREATE TABLE tableIdentifier #showCreateTable
| (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction
| (DESC | DESCRIBE) database EXTENDED? identifier #describeDatabase
| (DESC | DESCRIBE) database EXTENDED? db=errorCapturingIdentifier #describeDatabase
| (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)?
tableIdentifier partitionSpec? describeColName? #describeTable
| (DESC | DESCRIBE) QUERY? query #describeQuery
Expand Down Expand Up @@ -319,7 +322,7 @@ ctes
;

namedQuery
: name=identifier (columnAliases=identifierList)? AS? '(' query ')'
: name=errorCapturingIdentifier (columnAliases=identifierList)? AS? '(' query ')'
;

tableProvider
Expand Down Expand Up @@ -559,15 +562,15 @@ identifierList
;

identifierSeq
: identifier (',' identifier)*
: ident+=errorCapturingIdentifier (',' ident+=errorCapturingIdentifier)*
;

orderedIdentifierList
: '(' orderedIdentifier (',' orderedIdentifier)* ')'
;

orderedIdentifier
: identifier ordering=(ASC | DESC)?
: ident=errorCapturingIdentifier ordering=(ASC | DESC)?
;

identifierCommentList
Expand All @@ -591,7 +594,7 @@ inlineTable
;

functionTable
: identifier '(' (expression (',' expression)*)? ')' tableAlias
: funcName=errorCapturingIdentifier '(' (expression (',' expression)*)? ')' tableAlias
;

tableAlias
Expand All @@ -609,19 +612,19 @@ rowFormat
;

multipartIdentifier
: parts+=identifier ('.' parts+=identifier)*
: parts+=errorCapturingIdentifier ('.' parts+=errorCapturingIdentifier)*
;

tableIdentifier
: (db=identifier '.')? table=identifier
: (db=errorCapturingIdentifier '.')? table=errorCapturingIdentifier
;

functionIdentifier
: (db=identifier '.')? function=identifier
: (db=errorCapturingIdentifier '.')? function=errorCapturingIdentifier
;

namedExpression
: expression (AS? (identifier | identifierList))?
: expression (AS? (name=errorCapturingIdentifier | identifierList))?
;

namedExpressionSeq
Expand Down Expand Up @@ -788,7 +791,7 @@ colTypeList
;

colType
: identifier dataType (COMMENT STRING)?
: colName=errorCapturingIdentifier dataType (COMMENT STRING)?
;

complexColTypeList
Expand All @@ -808,18 +811,18 @@ windowClause
;

namedWindow
: identifier AS windowSpec
: name=errorCapturingIdentifier AS windowSpec
;

windowSpec
: name=identifier #windowRef
| '('name=identifier')' #windowRef
: name=errorCapturingIdentifier #windowRef
| '('name=errorCapturingIdentifier')' #windowRef
| '('
( CLUSTER BY partition+=expression (',' partition+=expression)*
| ((PARTITION | DISTRIBUTE) BY partition+=expression (',' partition+=expression)*)?
((ORDER | SORT) BY sortItem (',' sortItem)*)?)
windowFrame?
')' #windowDef
')' #windowDef
;

windowFrame
Expand All @@ -843,6 +846,19 @@ qualifiedName
: identifier ('.' identifier)*
;

// this rule is used for explicitly capturing wrong identifiers such as test-table, which should actually be `test-table`
// replace identifier with errorCapturingIdentifier where the immediate follow symbol is not an expression, otherwise
// valid expressions such as "a-b" can be recognized as an identifier
errorCapturingIdentifier
: identifier errorCapturingIdentifierExtra
;

// extra left-factoring grammar
errorCapturingIdentifierExtra
: (MINUS identifier)+ #errorIdent
| #realIdent
;

identifier
: strictIdentifier
| {!ansi}? strictNonReserved
Expand Down
Expand Up @@ -665,7 +665,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
// Collect all window specifications defined in the WINDOW clause.
val baseWindowMap = ctx.namedWindow.asScala.map {
wCtx =>
(wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec))
(wCtx.name.getText, typedVisit[WindowSpec](wCtx.windowSpec))
}.toMap

// Handle cases like
Expand Down Expand Up @@ -927,7 +927,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

val tvf = UnresolvedTableValuedFunction(
func.identifier.getText, func.expression.asScala.map(expression), aliases)
func.funcName.getText, func.expression.asScala.map(expression), aliases)
tvf.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan)
}

Expand Down Expand Up @@ -1026,7 +1026,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
* Create a Sequence of Strings for an identifier list.
*/
override def visitIdentifierSeq(ctx: IdentifierSeqContext): Seq[String] = withOrigin(ctx) {
ctx.identifier.asScala.map(_.getText)
ctx.ident.asScala.map(_.getText)
}

/* ********************************************************************************************
Expand Down Expand Up @@ -1086,8 +1086,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
*/
override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) {
val e = expression(ctx.expression)
if (ctx.identifier != null) {
Alias(e, ctx.identifier.getText)()
if (ctx.name != null) {
Alias(e, ctx.name.getText)()
} else if (ctx.identifierList != null) {
MultiAlias(e, visitIdentifierList(ctx.identifierList))
} else {
Expand Down Expand Up @@ -1479,7 +1479,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
* Create a reference to a window frame, i.e. [[WindowSpecReference]].
*/
override def visitWindowRef(ctx: WindowRefContext): WindowSpecReference = withOrigin(ctx) {
WindowSpecReference(ctx.identifier.getText)
WindowSpecReference(ctx.name.getText)
}

/**
Expand Down Expand Up @@ -1958,7 +1958,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}

StructField(
identifier.getText,
colName.getText,
cleanedDataType,
nullable = true,
builder.build())
Expand Down Expand Up @@ -2012,7 +2012,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging
}
}

orderedIdCtx.identifier.getText
orderedIdCtx.ident.getText
})
}

Expand Down
Expand Up @@ -265,6 +265,14 @@ class ParseException(
*/
case object PostProcessor extends SqlBaseBaseListener {

/** Throws error message when exiting a explicitly captured wrong identifier rule */
override def exitErrorIdent(ctx: SqlBaseParser.ErrorIdentContext): Unit = {
val ident = ctx.getParent.getText

throw new ParseException(s"Possibly unquoted identifier $ident detected. " +
s"Please consider quoting it with back-quotes as `$ident`", ctx)
}

/** Remove the back ticks from an Identifier. */
override def exitQuotedIdentifier(ctx: SqlBaseParser.QuotedIdentifierContext): Unit = {
replaceTokenByIdentifier(ctx, 1) { token =>
Expand Down
Expand Up @@ -17,11 +17,20 @@
package org.apache.spark.sql.catalyst.parser

import org.apache.spark.sql.catalyst.analysis.AnalysisTest
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan

/**
* Test various parser errors.
*/
class ErrorParserSuite extends AnalysisTest {
import CatalystSqlParser._
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._

private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = {
assert(parsePlan(sqlCommand) == plan)
}

def intercept(sqlCommand: String, messages: String*): Unit =
interceptParseException(CatalystSqlParser.parsePlan)(sqlCommand, messages: _*)

Expand Down Expand Up @@ -72,4 +81,105 @@ class ErrorParserSuite extends AnalysisTest {
intercept("select * from test where test.t is like 'test'", "mismatched input 'is' expecting")
intercept("SELECT * FROM test WHERE x NOT NULL", "mismatched input 'NOT' expecting")
}

test("hyphen in identifier - DDL tests") {
val msg = "unquoted identifier"
intercept("USE test-test", 1, 8, 9, msg + " test-test")
intercept("CREATE DATABASE IF NOT EXISTS my-database", 1, 32, 33, msg + " my-database")
intercept(
"""
|ALTER DATABASE my-database
|SET DBPROPERTIES ('p1'='v1')""".stripMargin, 2, 17, 18, msg + " my-database")
intercept("DROP DATABASE my-database", 1, 16, 17, msg + " my-database")
intercept(
"""
|ALTER TABLE t
|CHANGE COLUMN
|test-col BIGINT
""".stripMargin, 4, 4, 5, msg + " test-col")
intercept("CREATE TABLE test (attri-bute INT)", 1, 24, 25, msg + " attri-bute")
intercept(
"""
|CREATE TABLE IF NOT EXISTS mydb.page-view
|USING parquet
|COMMENT 'This is the staging page view table'
|LOCATION '/user/external/page_view'
|TBLPROPERTIES ('p1'='v1', 'p2'='v2')
|AS SELECT * FROM src""".stripMargin, 2, 36, 37, msg + " page-view")
intercept("SHOW TABLES IN hyphen-database", 1, 21, 22, msg + " hyphen-database")
intercept("SHOW TABLE EXTENDED IN hyphen-db LIKE \"str\"", 1, 29, 30, msg + " hyphen-db")
intercept("SHOW COLUMNS IN t FROM test-db", 1, 27, 28, msg + " test-db")
intercept("DESC SCHEMA EXTENDED test-db", 1, 25, 26, msg + " test-db")
intercept("ANALYZE TABLE test-table PARTITION (part1)", 1, 18, 19, msg + " test-table")
intercept("LOAD DATA INPATH \"path\" INTO TABLE my-tab", 1, 37, 38, msg + " my-tab")
}

test("hyphen in identifier - DML tests") {
val msg = "unquoted identifier"
// dml tests
intercept("SELECT * FROM table-with-hyphen", 1, 19, 25, msg + " table-with-hyphen")
// special test case: minus in expression shouldn't be treated as hyphen in identifiers
intercept("SELECT a-b FROM table-with-hyphen", 1, 21, 27, msg + " table-with-hyphen")
intercept("SELECT a-b AS a-b FROM t", 1, 15, 16, msg + " a-b")
intercept("SELECT a-b FROM table-hyphen WHERE a-b = 0", 1, 21, 22, msg + " table-hyphen")
intercept("SELECT (a - test_func(b-c)) FROM test-table", 1, 37, 38, msg + " test-table")
intercept("WITH a-b AS (SELECT 1 FROM s) SELECT * FROM s;", 1, 6, 7, msg + " a-b")
intercept(
"""
|SELECT a, b
|FROM t1 JOIN t2
|USING (a, b, at-tr)
""".stripMargin, 4, 15, 16, msg + " at-tr"
)
intercept(
"""
|SELECT product, category, dense_rank()
|OVER (PARTITION BY category ORDER BY revenue DESC) as hyphen-rank
|FROM productRevenue
""".stripMargin, 3, 60, 61, msg + " hyphen-rank"
)
intercept(
"""
|SELECT a, b
|FROM grammar-breaker
|WHERE a-b > 10
|GROUP BY fake-breaker
|ORDER BY c
""".stripMargin, 3, 12, 13, msg + " grammar-breaker")
assertEqual(
"""
|SELECT a, b
|FROM t
|WHERE a-b > 10
|GROUP BY fake-breaker
|ORDER BY c
""".stripMargin,
table("t")
.where('a - 'b > 10)
.groupBy('fake - 'breaker)('a, 'b)
.orderBy('c.asc))
intercept(
"""
|SELECT * FROM tab
|WINDOW hyphen-window AS
| (PARTITION BY a, b ORDER BY c rows BETWEEN 1 PRECEDING AND 1 FOLLOWING)
""".stripMargin, 3, 13, 14, msg + " hyphen-window")
intercept(
"""
|SELECT * FROM tab
|WINDOW window_ref AS window-ref
""".stripMargin, 3, 27, 28, msg + " window-ref")
intercept(
"""
|SELECT tb.*
|FROM t-a INNER JOIN tb
|ON ta.a = tb.a AND ta.tag = tb.tag
""".stripMargin, 3, 6, 7, msg + " t-a")
intercept(
"""
|FROM test-table
|SELECT a
|SELECT b
""".stripMargin, 2, 9, 10, msg + " test-table")
}
}

0 comments on commit 7b7f16f

Please sign in to comment.