Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-11972

[Spark SQL] the value of 'hiveconf' parameter in CLI can't be got after enter spark-sql session

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 1.6.0
    • 1.6.1, 2.0.0
    • SQL
    • None

    Description

      Reproduce Steps:
      /usr/lib/spark/bin/spark-sql -v --driver-memory 4g --executor-memory 7g --executor-cores 5 --num-executors 31 --master yarn-client --conf spark.yarn.executor.memoryOverhead=1024 --hiveconf RESULT_TABLE=test_result01

      >use test;
      >DROP TABLE IF EXISTS ${hiveconf:RESULT_TABLE};
      15/11/24 13:45:12 INFO parse.ParseDriver: Parsing command: DROP TABLE IF EXISTS ${hiveconf:RESULT_TABLE}
      NoViableAltException(16@[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.spark.sql.hive.HiveQl$.getAst(HiveQl.scala:276)
              at org.apache.spark.sql.hive.HiveQl$.createPlan(HiveQl.scala:303)
              at org.apache.spark.sql.hive.ExtendedHiveQlParser$$anonfun$hiveQl$1.apply(ExtendedHiveQlParser.scala:41)
              at org.apache.spark.sql.hive.ExtendedHiveQlParser$$anonfun$hiveQl$1.apply(ExtendedHiveQlParser.scala:40)
              at scala.util.parsing.combinator.Parsers$Success.map(Parsers.scala:136)
              at scala.util.parsing.combinator.Parsers$Success.map(Parsers.scala:135)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$map$1.apply(Parsers.scala:242)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$map$1.apply(Parsers.scala:242)
              at scala.util.parsing.combinator.Parsers$$anon$3.apply(Parsers.scala:222)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$append$1$$anonfun$apply$2.apply(Parsers.scala:254)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$append$1$$anonfun$apply$2.apply(Parsers.scala:254)
              at scala.util.parsing.combinator.Parsers$Failure.append(Parsers.scala:202)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$append$1.apply(Parsers.scala:254)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$append$1.apply(Parsers.scala:254)
              at scala.util.parsing.combinator.Parsers$$anon$3.apply(Parsers.scala:222)
              at scala.util.parsing.combinator.Parsers$$anon$2$$anonfun$apply$14.apply(Parsers.scala:891)
              at scala.util.parsing.combinator.Parsers$$anon$2$$anonfun$apply$14.apply(Parsers.scala:891)
              at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57)
              at scala.util.parsing.combinator.Parsers$$anon$2.apply(Parsers.scala:890)
              at scala.util.parsing.combinator.PackratParsers$$anon$1.apply(PackratParsers.scala:110)
              at org.apache.spark.sql.catalyst.AbstractSparkSQLParser.parse(AbstractSparkSQLParser.scala:34)
              at org.apache.spark.sql.hive.HiveQl$.parseSql(HiveQl.scala:295)
              at org.apache.spark.sql.hive.HiveQLDialect$$anonfun$parse$1.apply(HiveContext.scala:65)
              at org.apache.spark.sql.hive.HiveQLDialect$$anonfun$parse$1.apply(HiveContext.scala:65)
              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.HiveQLDialect.parse(HiveContext.scala:64)
              at org.apache.spark.sql.SQLContext$$anonfun$2.apply(SQLContext.scala:211)
              at org.apache.spark.sql.SQLContext$$anonfun$2.apply(SQLContext.scala:211)
              at org.apache.spark.sql.SparkSQLParser$$anonfun$org$apache$spark$sql$SparkSQLParser$$others$1.apply(SparkSQLParser.scala:115)
              at org.apache.spark.sql.SparkSQLParser$$anonfun$org$apache$spark$sql$SparkSQLParser$$others$1.apply(SparkSQLParser.scala:114)
              at scala.util.parsing.combinator.Parsers$Success.map(Parsers.scala:136)
              at scala.util.parsing.combinator.Parsers$Success.map(Parsers.scala:135)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$map$1.apply(Parsers.scala:242)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$map$1.apply(Parsers.scala:242)
              at scala.util.parsing.combinator.Parsers$$anon$3.apply(Parsers.scala:222)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$append$1$$anonfun$apply$2.apply(Parsers.scala:254)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$append$1$$anonfun$apply$2.apply(Parsers.scala:254)
              at scala.util.parsing.combinator.Parsers$Failure.append(Parsers.scala:202)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$append$1.apply(Parsers.scala:254)
              at scala.util.parsing.combinator.Parsers$Parser$$anonfun$append$1.apply(Parsers.scala:254)
              at scala.util.parsing.combinator.Parsers$$anon$3.apply(Parsers.scala:222)
              at scala.util.parsing.combinator.Parsers$$anon$2$$anonfun$apply$14.apply(Parsers.scala:891)
              at scala.util.parsing.combinator.Parsers$$anon$2$$anonfun$apply$14.apply(Parsers.scala:891)
              at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57)
              at scala.util.parsing.combinator.Parsers$$anon$2.apply(Parsers.scala:890)
              at scala.util.parsing.combinator.PackratParsers$$anon$1.apply(PackratParsers.scala:110)
              at org.apache.spark.sql.catalyst.AbstractSparkSQLParser.parse(AbstractSparkSQLParser.scala:34)
              at org.apache.spark.sql.SQLContext$$anonfun$1.apply(SQLContext.scala:208)
              at org.apache.spark.sql.SQLContext$$anonfun$1.apply(SQLContext.scala:208)
              at org.apache.spark.sql.execution.datasources.DDLParser.parse(DDLParser.scala:43)
              at org.apache.spark.sql.SQLContext.parseSql(SQLContext.scala:231)
              at org.apache.spark.sql.hive.HiveContext.parseSql(HiveContext.scala:329)
              at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:808)
              at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:63)
              at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:308)
              at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376)
              at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:226)
              at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
              at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
              at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.lang.reflect.Method.invoke(Method.java:606)
              at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:727)
              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)
      Error in query: cannot recognize input near '$' '{' 'hiveconf' in table name; line 1 pos 21
      

      Attachments

        Issue Links

          Activity

            People

              adrian-wang Adrian Wang
              jameszhouyi Yi Zhou
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: