From 65c941d3ac31c0c603e7318ea849205183d5a66e Mon Sep 17 00:00:00 2001 From: ClassNotFoundExp Date: Thu, 10 Nov 2016 16:51:11 +0800 Subject: [PATCH] cannot create table by using the hive default fileformat --- .../spark/sql/execution/SparkSqlParser.scala | 17 +++++++++++++---- .../spark/sql/internal/SessionState.scala | 2 +- 2 files changed, 14 insertions(+), 5 deletions(-) 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 b8be3d17ba444..0f585a8529bb4 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 @@ -22,6 +22,8 @@ import scala.collection.JavaConverters._ import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.TerminalNode +import org.apache.hadoop.conf.Configuration + import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ @@ -36,8 +38,9 @@ import org.apache.spark.sql.types.StructType /** * Concrete parser for Spark SQL statements. */ -class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser { - val astBuilder = new SparkSqlAstBuilder(conf) +class SparkSqlParser(conf: SQLConf, + hadoopConf: Option[Configuration] = None) extends AbstractSqlParser { + val astBuilder = new SparkSqlAstBuilder(conf, hadoopConf) private val substitutor = new VariableSubstitution(conf) @@ -49,7 +52,8 @@ class SparkSqlParser(conf: SQLConf) extends AbstractSqlParser { /** * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier. */ -class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { +class SparkSqlAstBuilder(conf: SQLConf, + hadoopConf: Option[Configuration] = None) extends AstBuilder { import org.apache.spark.sql.catalyst.parser.ParserUtils._ /** @@ -991,7 +995,12 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { // Storage format val defaultStorage: CatalogStorageFormat = { - val defaultStorageType = conf.getConfString("hive.default.fileformat", "textfile") + val defaultStorageType = if (hadoopConf.isDefined) { + hadoopConf.get.get("hive.default.fileformat", "textfile") + } else { + conf.getConfString("hive.default.fileformat", "textfile") + } + val defaultHiveSerde = HiveSerDe.sourceToSerDe(defaultStorageType) CatalogStorageFormat( locationUri = None, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 8759dfe39ce1d..e9bf4f8f1cbcb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -132,7 +132,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { /** * Parser that extracts expressions, plans, table identifiers etc. from SQL texts. */ - lazy val sqlParser: ParserInterface = new SparkSqlParser(conf) + lazy val sqlParser: ParserInterface = new SparkSqlParser(conf, Some(newHadoopConf())) /** * Planner that converts optimized logical plans to physical plans.