This is an automated email from the ASF dual-hosted git repository.

lixiao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 7b7f16f  [SPARK-27890][SQL] Improve SQL parser error message for 
character-only identifier with hyphens except those in expressions
7b7f16f is described below

commit 7b7f16f2a7a6a6685a89176666a9b5ba403fff76
Author: Yesheng Ma <kimi.y...@gmail.com>
AuthorDate: Tue Jun 18 21:51:15 2019 -0700

    [SPARK-27890][SQL] Improve SQL parser error message for character-only 
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.y...@gmail.com>
    Signed-off-by: gatorsmile <gatorsm...@gmail.com>
---
 .../apache/spark/sql/catalyst/parser/SqlBase.g4    |  60 ++++++-----
 .../spark/sql/catalyst/parser/AstBuilder.scala     |  16 +--
 .../spark/sql/catalyst/parser/ParseDriver.scala    |   8 ++
 .../sql/catalyst/parser/ErrorParserSuite.scala     | 110 +++++++++++++++++++++
 .../spark/sql/execution/SparkSqlParser.scala       |  10 +-
 5 files changed, 169 insertions(+), 35 deletions(-)

diff --git 
a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 
b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index dcb7939..f57a659 100644
--- 
a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ 
b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -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) |
@@ -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)?
@@ -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
@@ -319,7 +322,7 @@ ctes
     ;
 
 namedQuery
-    : name=identifier (columnAliases=identifierList)? AS? '(' query ')'
+    : name=errorCapturingIdentifier (columnAliases=identifierList)? AS? '(' 
query ')'
     ;
 
 tableProvider
@@ -559,7 +562,7 @@ identifierList
     ;
 
 identifierSeq
-    : identifier (',' identifier)*
+    : ident+=errorCapturingIdentifier (',' ident+=errorCapturingIdentifier)*
     ;
 
 orderedIdentifierList
@@ -567,7 +570,7 @@ orderedIdentifierList
     ;
 
 orderedIdentifier
-    : identifier ordering=(ASC | DESC)?
+    : ident=errorCapturingIdentifier ordering=(ASC | DESC)?
     ;
 
 identifierCommentList
@@ -591,7 +594,7 @@ inlineTable
     ;
 
 functionTable
-    : identifier '(' (expression (',' expression)*)? ')' tableAlias
+    : funcName=errorCapturingIdentifier '(' (expression (',' expression)*)? 
')' tableAlias
     ;
 
 tableAlias
@@ -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
@@ -788,7 +791,7 @@ colTypeList
     ;
 
 colType
-    : identifier dataType (COMMENT STRING)?
+    : colName=errorCapturingIdentifier dataType (COMMENT STRING)?
     ;
 
 complexColTypeList
@@ -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
@@ -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
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index e0e5d42..c7db259 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -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
@@ -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)
   }
 
@@ -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)
   }
 
   /* 
********************************************************************************************
@@ -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 {
@@ -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)
   }
 
   /**
@@ -1958,7 +1958,7 @@ class AstBuilder(conf: SQLConf) extends 
SqlBaseBaseVisitor[AnyRef] with Logging
     }
 
     StructField(
-      identifier.getText,
+      colName.getText,
       cleanedDataType,
       nullable = true,
       builder.build())
@@ -2012,7 +2012,7 @@ class AstBuilder(conf: SQLConf) extends 
SqlBaseBaseVisitor[AnyRef] with Logging
               }
             }
 
-            orderedIdCtx.identifier.getText
+            orderedIdCtx.ident.getText
           })
   }
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
index 1cc12d9..932e795 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
@@ -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 =>
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
index a2b83c8..478953f 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
@@ -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: 
_*)
 
@@ -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")
+  }
 }
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
index de1fbc0..12cd8ab 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
@@ -510,7 +510,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends 
AstBuilder(conf) {
     checkDuplicateClauses(ctx.DBPROPERTIES, "WITH DBPROPERTIES", ctx)
 
     CreateDatabaseCommand(
-      ctx.identifier.getText,
+      ctx.db.getText,
       ctx.EXISTS != null,
       ctx.locationSpec.asScala.headOption.map(visitLocationSpec),
       Option(ctx.comment).map(string),
@@ -528,7 +528,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends 
AstBuilder(conf) {
   override def visitSetDatabaseProperties(
       ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) {
     AlterDatabasePropertiesCommand(
-      ctx.identifier.getText,
+      ctx.db.getText,
       visitPropertyKeyValues(ctx.tablePropertyList))
   }
 
@@ -541,7 +541,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends 
AstBuilder(conf) {
    * }}}
    */
   override def visitDropDatabase(ctx: DropDatabaseContext): LogicalPlan = 
withOrigin(ctx) {
-    DropDatabaseCommand(ctx.identifier.getText, ctx.EXISTS != null, 
ctx.CASCADE != null)
+    DropDatabaseCommand(ctx.db.getText, ctx.EXISTS != null, ctx.CASCADE != 
null)
   }
 
   /**
@@ -553,7 +553,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends 
AstBuilder(conf) {
    * }}}
    */
   override def visitDescribeDatabase(ctx: DescribeDatabaseContext): 
LogicalPlan = withOrigin(ctx) {
-    DescribeDatabaseCommand(ctx.identifier.getText, ctx.EXTENDED != null)
+    DescribeDatabaseCommand(ctx.db.getText, ctx.EXTENDED != null)
   }
 
   /**
@@ -807,7 +807,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends 
AstBuilder(conf) {
 
     AlterTableChangeColumnCommand(
       tableName = visitTableIdentifier(ctx.tableIdentifier),
-      columnName = ctx.identifier.getText,
+      columnName = ctx.colName.getText,
       newColumn = visitColType(ctx.colType))
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to