[ 
https://issues.apache.org/jira/browse/SPARK-13454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Apache Spark reassigned SPARK-13454:
------------------------------------

    Assignee: Apache Spark

> Cannot drop table whose name starts with underscore
> ---------------------------------------------------
>
>                 Key: SPARK-13454
>                 URL: https://issues.apache.org/jira/browse/SPARK-13454
>             Project: Spark
>          Issue Type: Bug
>          Components: SQL
>    Affects Versions: 1.6.0, 1.6.1
>            Reporter: Cheng Lian
>            Assignee: Apache Spark
>
> Spark shell snippet for reproduction:
> {code}
> sqlContext.sql("CREATE TABLE `_a`(i INT)") // This one works.
> sqlContext.sql("DROP TABLE `_a`") // This one failed. Basically, we cannot 
> drop a table starting with _ in Spark 1.6.0. Master is fine.
> {code}
> Exception thrown:
> {noformat}
> NoViableAltException(13@[192:1: tableName : (db= identifier DOT tab= 
> identifier -> ^( TOK_TABNAME $db $tab) |tab= identifier -> ^( TOK_TABNAME 
> $tab) );])
>         at org.antlr.runtime.DFA.noViableAlt(DFA.java:158)
>         at org.antlr.runtime.DFA.predict(DFA.java:144)
>         at 
> org.apache.hadoop.hive.ql.parse.HiveParser_FromClauseParser.tableName(HiveParser_FromClauseParser.java:4747)
>         at 
> org.apache.hadoop.hive.ql.parse.HiveParser.tableName(HiveParser.java:45918)
>         at 
> org.apache.hadoop.hive.ql.parse.HiveParser.dropTableStatement(HiveParser.java:7133)
>         at 
> org.apache.hadoop.hive.ql.parse.HiveParser.ddlStatement(HiveParser.java:2655)
>         at 
> org.apache.hadoop.hive.ql.parse.HiveParser.execStatement(HiveParser.java:1650)
>         at 
> org.apache.hadoop.hive.ql.parse.HiveParser.statement(HiveParser.java:1109)
>         at 
> org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:202)
>         at 
> org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:166)
>         at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:396)
>         at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:308)
>         at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:1122)
>         at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1170)
>         at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1059)
>         at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1049)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:484)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:473)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:279)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.liftedTree1$1(ClientWrapper.scala:226)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:225)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:268)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:473)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:463)
>         at 
> org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:605)
>         at 
> org.apache.spark.sql.hive.execution.DropTable.run(commands.scala:73)
>         at 
> org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:58)
>         at 
> org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:56)
>         at 
> org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:70)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130)
>         at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)
>         at 
> org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130)
>         at 
> org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:55)
>         at 
> org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:55)
>         at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:145)
>         at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:130)
>         at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52)
>         at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:817)
>         at 
> $line21.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:26)
>         at 
> $line21.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:31)
>         at $line21.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:33)
>         at $line21.$read$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:35)
>         at $line21.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:37)
>         at $line21.$read$$iwC$$iwC$$iwC.<init>(<console>:39)
>         at $line21.$read$$iwC$$iwC.<init>(<console>:41)
>         at $line21.$read$$iwC.<init>(<console>:43)
>         at $line21.$read.<init>(<console>:45)
>         at $line21.$read$.<init>(<console>:49)
>         at $line21.$read$.<clinit>(<console>)
>         at $line21.$eval$.<init>(<console>:7)
>         at $line21.$eval$.<clinit>(<console>)
>         at $line21.$eval.$print(<console>)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:483)
>         at 
> org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)
>         at 
> org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1346)
>         at 
> org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)
>         at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)
>         at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)
>         at 
> org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:857)
>         at 
> org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:902)
>         at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:814)
>         at 
> org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:657)
>         at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:665)
>         at 
> org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:670)
>         at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:997)
>         at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
>         at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
>         at 
> scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
>         at 
> org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$process(SparkILoop.scala:945)
>         at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:1059)
>         at org.apache.spark.repl.Main$.main(Main.scala:31)
>         at org.apache.spark.repl.Main.main(Main.scala)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:483)
>         at 
> org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:731)
>         at 
> org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:181)
>         at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:206)
>         at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:121)
>         at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> 16/02/23 22:12:04 ERROR ql.Driver: FAILED: ParseException line 1:11 cannot 
> recognize input near '_a' '<EOF>' '<EOF>' in table name
> org.apache.hadoop.hive.ql.parse.ParseException: line 1:11 cannot recognize 
> input near '_a' '<EOF>' '<EOF>' in table name
>         at 
> org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:205)
>         at 
> org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:166)
>         at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:396)
>         at org.apache.hadoop.hive.ql.Driver.compile(Driver.java:308)
>         at org.apache.hadoop.hive.ql.Driver.compileInternal(Driver.java:1122)
>         at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1170)
>         at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1059)
>         at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1049)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:484)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:473)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:279)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.liftedTree1$1(ClientWrapper.scala:226)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:225)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:268)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:473)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:463)
>         at 
> org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:605)
>         at 
> org.apache.spark.sql.hive.execution.DropTable.run(commands.scala:73)
>         at 
> org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:58)
>         at 
> org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:56)
>         at 
> org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:70)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130)
>         at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)
>         at 
> org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130)
>         at 
> org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:55)
>         at 
> org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:55)
>         at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:145)
>         at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:130)
>         at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52)
>         at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:817)
>         at 
> $line21.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:26)
>         at 
> $line21.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:31)
>         at $line21.$read$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:33)
>         at $line21.$read$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:35)
>         at $line21.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:37)
>         at $line21.$read$$iwC$$iwC$$iwC.<init>(<console>:39)
>         at $line21.$read$$iwC$$iwC.<init>(<console>:41)
>         at $line21.$read$$iwC.<init>(<console>:43)
>         at $line21.$read.<init>(<console>:45)
>         at $line21.$read$.<init>(<console>:49)
>         at $line21.$read$.<clinit>(<console>)
>         at $line21.$eval$.<init>(<console>:7)
>         at $line21.$eval$.<clinit>(<console>)
>         at $line21.$eval.$print(<console>)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:483)
>         at 
> org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)
>         at 
> org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1346)
>         at 
> org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)
>         at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)
>         at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)
>         at 
> org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:857)
>         at 
> org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:902)
>         at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:814)
>         at 
> org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:657)
>         at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:665)
>         at 
> org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:670)
>         at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:997)
>         at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
>         at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
>         at 
> scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
>         at 
> org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$process(SparkILoop.scala:945)
>         at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:1059)
>         at org.apache.spark.repl.Main$.main(Main.scala:31)
>         at org.apache.spark.repl.Main.main(Main.scala)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:483)
>         at 
> org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:731)
>         at 
> org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:181)
>         at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:206)
>         at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:121)
>         at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> 16/02/23 22:12:04 INFO log.PerfLogger: </PERFLOG method=compile 
> start=1456236724085 end=1456236724093 duration=8 
> from=org.apache.hadoop.hive.ql.Driver>
> 16/02/23 22:12:04 INFO log.PerfLogger: <PERFLOG method=releaseLocks 
> from=org.apache.hadoop.hive.ql.Driver>
> 16/02/23 22:12:04 INFO log.PerfLogger: </PERFLOG method=releaseLocks 
> start=1456236724094 end=1456236724094 duration=0 
> from=org.apache.hadoop.hive.ql.Driver>
> 16/02/23 22:12:04 ERROR client.ClientWrapper:
> ======================
> HIVE FAILURE OUTPUT
> ======================
> SET hive.support.sql11.reserved.keywords=false
> FAILED: Execution Error, return code 1 from 
> org.apache.hadoop.hive.ql.exec.DDLTask. AlreadyExistsException(message:Table 
> _a already exists)
> FAILED: ParseException line 1:11 cannot recognize input near '_a' '<EOF>' 
> '<EOF>' in table name
> ======================
> END HIVE FAILURE OUTPUT
> ======================
> org.apache.spark.sql.execution.QueryExecutionException: FAILED: 
> ParseException line 1:11 cannot recognize input near '_a' '<EOF>' '<EOF>' in 
> table name
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:488)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:473)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:279)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.liftedTree1$1(ClientWrapper.scala:226)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:225)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:268)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:473)
>         at 
> org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:463)
>         at 
> org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:605)
>         at 
> org.apache.spark.sql.hive.execution.DropTable.run(commands.scala:73)
>         at 
> org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:58)
>         at 
> org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:56)
>         at 
> org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:70)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132)
>         at 
> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130)
>         at 
> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)
>         at 
> org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130)
>         at 
> org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:55)
>         at 
> org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:55)
>         at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:145)
>         at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:130)
>         at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52)
>         at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:817)
>         at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:26)
>         at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:31)
>         at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:33)
>         at $iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:35)
>         at $iwC$$iwC$$iwC$$iwC.<init>(<console>:37)
>         at $iwC$$iwC$$iwC.<init>(<console>:39)
>         at $iwC$$iwC.<init>(<console>:41)
>         at $iwC.<init>(<console>:43)
>         at <init>(<console>:45)
>         at .<init>(<console>:49)
>         at .<clinit>(<console>)
>         at .<init>(<console>:7)
>         at .<clinit>(<console>)
>         at $print(<console>)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:483)
>         at 
> org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065)
>         at 
> org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1346)
>         at 
> org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840)
>         at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871)
>         at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819)
>         at 
> org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:857)
>         at 
> org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:902)
>         at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:814)
>         at 
> org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:657)
>         at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:665)
>         at 
> org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:670)
>         at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:997)
>         at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
>         at 
> org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945)
>         at 
> scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
>         at 
> org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$process(SparkILoop.scala:945)
>         at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:1059)
>         at org.apache.spark.repl.Main$.main(Main.scala:31)
>         at org.apache.spark.repl.Main.main(Main.scala)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:483)
>         at 
> org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:731)
>         at 
> org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:181)
>         at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:206)
>         at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:121)
>         at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> {noformat}
> The reason is that, after being parsed, {{DROP TABLE}} is firstly handled by 
> {{DropTable}}. Then, {{DropTable}} constructs a new {{DROP TABLE}} SQL 
> statement and delegate it to Hive. However, the table name in the constructed 
> SQL statement isn't quoted while Hive lexer doesn't allow unquoted 
> identifiers to start with underscore.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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

Reply via email to