Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand All @@ -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)

Expand All @@ -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._

/**
Expand Down Expand Up @@ -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,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down