diff options
author | Reynold Xin <rxin@databricks.com> | 2016-04-20 21:20:51 -0700 |
---|---|---|
committer | Yin Huai <yhuai@databricks.com> | 2016-04-20 21:20:51 -0700 |
commit | 8045814114d41dc17130ebc71f9e48d28ca959ca (patch) | |
tree | 923bd39a5ca5feac519b4da04296beb89ce25d28 /sql/hive/src/main/scala/org | |
parent | 90933e2afaec170b0f82ef61a443d69c40f4cda5 (diff) | |
download | spark-8045814114d41dc17130ebc71f9e48d28ca959ca.tar.gz spark-8045814114d41dc17130ebc71f9e48d28ca959ca.tar.bz2 spark-8045814114d41dc17130ebc71f9e48d28ca959ca.zip |
[SPARK-14782][SPARK-14778][SQL] Remove HiveConf dependency from HiveSqlAstBuilder
## What changes were proposed in this pull request?
The patch removes HiveConf dependency from HiveSqlAstBuilder. This is required in order to merge HiveSqlParser and SparkSqlAstBuilder, which would require getting rid of the Hive specific dependencies in HiveSqlParser.
This patch also accomplishes [SPARK-14778] Remove HiveSessionState.substitutor.
## How was this patch tested?
This should be covered by existing tests.
Author: Reynold Xin <rxin@databricks.com>
Closes #12550 from rxin/SPARK-14782.
Diffstat (limited to 'sql/hive/src/main/scala/org')
3 files changed, 26 insertions, 39 deletions
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 33a926e4d2..c20b022e84 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -24,7 +24,6 @@ import com.google.common.base.Objects import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata.{Table => HiveTable, _} @@ -46,6 +45,7 @@ import org.apache.spark.sql.execution.datasources.parquet.{DefaultSource => Parq import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.hive.execution.HiveNativeCommand import org.apache.spark.sql.hive.orc.{DefaultSource => OrcDefaultSource} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ private[hive] case class HiveSerDe( @@ -59,10 +59,10 @@ private[hive] object HiveSerDe { * * @param source Currently the source abbreviation can be one of the following: * SequenceFile, RCFile, ORC, PARQUET, and case insensitive. - * @param hiveConf Hive Conf + * @param conf SQLConf * @return HiveSerDe associated with the specified source */ - def sourceToSerDe(source: String, hiveConf: HiveConf): Option[HiveSerDe] = { + def sourceToSerDe(source: String, conf: SQLConf): Option[HiveSerDe] = { val serdeMap = Map( "sequencefile" -> HiveSerDe( @@ -73,7 +73,8 @@ private[hive] object HiveSerDe { HiveSerDe( inputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileInputFormat"), outputFormat = Option("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"), - serde = Option(hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTRCFILESERDE))), + serde = Option(conf.getConfString("hive.default.rcfile.serde", + "org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"))), "orc" -> HiveSerDe( @@ -297,7 +298,7 @@ private[hive] class HiveMetastoreCatalog(hive: SQLContext) extends Logging { CatalogTableType.MANAGED_TABLE } - val maybeSerDe = HiveSerDe.sourceToSerDe(provider, hiveconf) + val maybeSerDe = HiveSerDe.sourceToSerDe(provider, conf) val dataSource = DataSource( hive, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 09297c27dc..2c360cb7ca 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -53,11 +53,6 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) */ lazy val metadataHive: HiveClient = sharedState.metadataHive.newSession() - /** - * A Hive helper class for substituting variables in a SQL statement. - */ - lazy val substitutor = new VariableSubstitution - override lazy val conf: SQLConf = new SQLConf { override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false) } @@ -114,7 +109,7 @@ private[hive] class HiveSessionState(ctx: SQLContext) extends SessionState(ctx) /** * Parser for HiveQl query texts. */ - override lazy val sqlParser: ParserInterface = new HiveSqlParser(substitutor, hiveconf) + override lazy val sqlParser: ParserInterface = new HiveSqlParser(conf, hiveconf) /** * Planner that takes into account Hive-specific strategies. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala index 4ff02cdbd0..90f10d5ebd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveSqlParser.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConverters._ +import scala.util.Try import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.parse.{EximUtil, VariableSubstitution} +import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser._ @@ -32,18 +31,16 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkSqlAstBuilder import org.apache.spark.sql.execution.command.{CreateTable, CreateTableLike} import org.apache.spark.sql.hive.{CreateTableAsSelect => CTAS, CreateViewAsSelect => CreateView, HiveSerDe} -import org.apache.spark.sql.hive.{HiveGenericUDTF, HiveMetastoreTypes, HiveSerDe} -import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper +import org.apache.spark.sql.internal.SQLConf /** * Concrete parser for HiveQl statements. */ -class HiveSqlParser( - substitutor: VariableSubstitution, - hiveconf: HiveConf) - extends AbstractSqlParser { +class HiveSqlParser(conf: SQLConf, hiveconf: HiveConf) extends AbstractSqlParser { - val astBuilder = new HiveSqlAstBuilder(hiveconf) + val astBuilder = new HiveSqlAstBuilder(conf) + + lazy val substitutor = new VariableSubstitution protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = { super.parse(substitutor.substitute(hiveconf, command))(toResult) @@ -57,7 +54,7 @@ class HiveSqlParser( /** * Builder that converts an ANTLR ParseTree into a LogicalPlan/Expression/TableIdentifier. */ -class HiveSqlAstBuilder(hiveConf: HiveConf) extends SparkSqlAstBuilder { +class HiveSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder { import ParserUtils._ /** @@ -184,8 +181,8 @@ class HiveSqlAstBuilder(hiveConf: HiveConf) extends SparkSqlAstBuilder { // Storage format val defaultStorage: CatalogStorageFormat = { - val defaultStorageType = hiveConf.getVar(HiveConf.ConfVars.HIVEDEFAULTFILEFORMAT) - val defaultHiveSerde = HiveSerDe.sourceToSerDe(defaultStorageType, hiveConf) + val defaultStorageType = conf.getConfString("hive.default.fileformat", "textfile") + val defaultHiveSerde = HiveSerDe.sourceToSerDe(defaultStorageType, conf) CatalogStorageFormat( locationUri = None, inputFormat = defaultHiveSerde.flatMap(_.inputFormat) @@ -323,7 +320,7 @@ class HiveSqlAstBuilder(hiveConf: HiveConf) extends SparkSqlAstBuilder { // Decode and input/output format. type Format = (Seq[(String, String)], Option[String], Seq[(String, String)], Option[String]) - def format(fmt: RowFormatContext, confVar: ConfVars): Format = fmt match { + def format(fmt: RowFormatContext, configKey: String): Format = fmt match { case c: RowFormatDelimitedContext => // TODO we should use the visitRowFormatDelimited function here. However HiveScriptIOSchema // expects a seq of pairs in which the old parsers' token names are used as keys. @@ -345,8 +342,8 @@ class HiveSqlAstBuilder(hiveConf: HiveConf) extends SparkSqlAstBuilder { val CatalogStorageFormat(None, None, None, Some(name), props) = visitRowFormatSerde(c) // SPARK-10310: Special cases LazySimpleSerDe - val recordHandler = if (name == classOf[LazySimpleSerDe].getCanonicalName) { - Option(hiveConf.getVar(confVar)) + val recordHandler = if (name == "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") { + Try(conf.getConfString(configKey)).toOption } else { None } @@ -354,17 +351,18 @@ class HiveSqlAstBuilder(hiveConf: HiveConf) extends SparkSqlAstBuilder { case null => // Use default (serde) format. - val name = hiveConf.getVar(ConfVars.HIVESCRIPTSERDE) + val name = conf.getConfString("hive.script.serde", + "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") val props = Seq(serdeConstants.FIELD_DELIM -> "\t") - val recordHandler = Option(hiveConf.getVar(confVar)) + val recordHandler = Try(conf.getConfString(configKey)).toOption (Nil, Option(name), props, recordHandler) } val (inFormat, inSerdeClass, inSerdeProps, reader) = - format(inRowFormat, ConfVars.HIVESCRIPTRECORDREADER) + format(inRowFormat, "hive.script.recordreader") val (outFormat, outSerdeClass, outSerdeProps, writer) = - format(inRowFormat, ConfVars.HIVESCRIPTRECORDWRITER) + format(outRowFormat, "hive.script.recordwriter") HiveScriptIOSchema( inFormat, outFormat, @@ -374,13 +372,6 @@ class HiveSqlAstBuilder(hiveConf: HiveConf) extends SparkSqlAstBuilder { schemaLess) } - /** - * Create location string. - */ - override def visitLocationSpec(ctx: LocationSpecContext): String = { - EximUtil.relativeToAbsolutePath(hiveConf, super.visitLocationSpec(ctx)) - } - /** Empty storage format for default values and copies. */ private val EmptyStorageFormat = CatalogStorageFormat(None, None, None, None, Map.empty) @@ -402,7 +393,7 @@ class HiveSqlAstBuilder(hiveConf: HiveConf) extends SparkSqlAstBuilder { override def visitGenericFileFormat( ctx: GenericFileFormatContext): CatalogStorageFormat = withOrigin(ctx) { val source = ctx.identifier.getText - HiveSerDe.sourceToSerDe(source, hiveConf) match { + HiveSerDe.sourceToSerDe(source, conf) match { case Some(s) => EmptyStorageFormat.copy( inputFormat = s.inputFormat, |