New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-18243][SQL] Port Hive writing to use FileFormat interface #16517
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,14 +21,14 @@ import org.apache.spark.sql._ | |
import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.catalyst.planning._ | ||
import org.apache.spark.sql.catalyst.plans._ | ||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, ScriptTransformation} | ||
import org.apache.spark.sql.catalyst.rules.Rule | ||
import org.apache.spark.sql.execution._ | ||
import org.apache.spark.sql.execution.command.{DDLUtils, ExecutedCommandExec} | ||
import org.apache.spark.sql.execution.command.DDLUtils | ||
import org.apache.spark.sql.execution.datasources.CreateTable | ||
import org.apache.spark.sql.hive.execution._ | ||
import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} | ||
import org.apache.spark.sql.types.StructType | ||
|
||
|
||
/** | ||
|
@@ -86,6 +86,47 @@ class DetermineHiveSerde(conf: SQLConf) extends Rule[LogicalPlan] { | |
} | ||
} | ||
|
||
class HiveAnalysis(session: SparkSession) extends Rule[LogicalPlan] { | ||
override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { | ||
case InsertIntoTable(table: MetastoreRelation, partSpec, query, overwrite, ifNotExists) | ||
if hasBeenPreprocessed(table.output, table.partitionKeys.toStructType, partSpec, query) => | ||
InsertIntoHiveTable(table, partSpec, query, overwrite, ifNotExists) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. to use |
||
|
||
case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => | ||
// Currently `DataFrameWriter.saveAsTable` doesn't support the Append mode of hive serde | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the code block below is mostly moved from https://github.com/apache/spark/pull/16517/files#diff-c4ed9859978dd6ac271b6a40ee945e4bL112 |
||
// tables yet. | ||
if (mode == SaveMode.Append) { | ||
throw new AnalysisException( | ||
"CTAS for hive serde tables does not support append semantics.") | ||
} | ||
|
||
val dbName = tableDesc.identifier.database.getOrElse(session.catalog.currentDatabase) | ||
CreateHiveTableAsSelectCommand( | ||
tableDesc.copy(identifier = tableDesc.identifier.copy(database = Some(dbName))), | ||
query, | ||
mode == SaveMode.Ignore) | ||
} | ||
|
||
/** | ||
* Returns true if the [[InsertIntoTable]] plan has already been preprocessed by analyzer rule | ||
* [[PreprocessTableInsertion]]. It is important that this rule([[HiveAnalysis]]) has to | ||
* be run after [[PreprocessTableInsertion]], to normalize the column names in partition spec and | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This rule is in the same batch with PreprocessTableInsertion, right? If so, we cannot guarantee that PreprocessTableInsertion will always fire first for a command before InsertIntoTable. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Or, you mean that we use this function to determine if PreprocessTableInsertion has fired? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should this function actually be part of the resolved method of InsertIntoTable? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think this version is good for now. |
||
* fix the schema mismatch by adding Cast. | ||
*/ | ||
private def hasBeenPreprocessed( | ||
tableOutput: Seq[Attribute], | ||
partSchema: StructType, | ||
partSpec: Map[String, Option[String]], | ||
query: LogicalPlan): Boolean = { | ||
val partColNames = partSchema.map(_.name).toSet | ||
query.resolved && partSpec.keys.forall(partColNames.contains) && { | ||
val staticPartCols = partSpec.filter(_._2.isDefined).keySet | ||
val expectedColumns = tableOutput.filterNot(a => staticPartCols.contains(a.name)) | ||
expectedColumns.toStructType.sameType(query.schema) | ||
} | ||
} | ||
} | ||
|
||
private[hive] trait HiveStrategies { | ||
// Possibly being too clever with types here... or not clever enough. | ||
self: SparkPlanner => | ||
|
@@ -94,35 +135,9 @@ private[hive] trait HiveStrategies { | |
|
||
object Scripts extends Strategy { | ||
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { | ||
case logical.ScriptTransformation(input, script, output, child, ioschema) => | ||
case ScriptTransformation(input, script, output, child, ioschema) => | ||
val hiveIoSchema = HiveScriptIOSchema(ioschema) | ||
ScriptTransformation(input, script, output, planLater(child), hiveIoSchema) :: Nil | ||
case _ => Nil | ||
} | ||
} | ||
|
||
object DataSinks extends Strategy { | ||
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { | ||
case logical.InsertIntoTable( | ||
table: MetastoreRelation, partition, child, overwrite, ifNotExists) => | ||
InsertIntoHiveTable( | ||
table, partition, planLater(child), overwrite, ifNotExists) :: Nil | ||
|
||
case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => | ||
// Currently `DataFrameWriter.saveAsTable` doesn't support | ||
// the Append mode of hive serde tables yet. | ||
if (mode == SaveMode.Append) { | ||
throw new AnalysisException( | ||
"CTAS for hive serde tables does not support append semantics.") | ||
} | ||
|
||
val dbName = tableDesc.identifier.database.getOrElse(sparkSession.catalog.currentDatabase) | ||
val cmd = CreateHiveTableAsSelectCommand( | ||
tableDesc.copy(identifier = tableDesc.identifier.copy(database = Some(dbName))), | ||
query, | ||
mode == SaveMode.Ignore) | ||
ExecutedCommandExec(cmd) :: Nil | ||
|
||
ScriptTransformationExec(input, script, output, planLater(child), hiveIoSchema) :: Nil | ||
case _ => Nil | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,149 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.sql.hive.execution | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.hadoop.fs.{FileStatus, Path} | ||
import org.apache.hadoop.hive.ql.exec.Utilities | ||
import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} | ||
import org.apache.hadoop.hive.serde2.Serializer | ||
import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, StructObjectInspector} | ||
import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption | ||
import org.apache.hadoop.io.Writable | ||
import org.apache.hadoop.mapred.{JobConf, Reporter} | ||
import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} | ||
|
||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory} | ||
import org.apache.spark.sql.hive.{HiveInspectors, HiveTableUtil} | ||
import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} | ||
import org.apache.spark.sql.types.StructType | ||
import org.apache.spark.util.SerializableJobConf | ||
|
||
/** | ||
* `FileFormat` for writing Hive tables. | ||
* | ||
* TODO: implement the read logic. | ||
*/ | ||
class HiveFileFormat(fileSinkConf: FileSinkDesc) extends FileFormat with Logging { | ||
override def inferSchema( | ||
sparkSession: SparkSession, | ||
options: Map[String, String], | ||
files: Seq[FileStatus]): Option[StructType] = { | ||
throw new UnsupportedOperationException(s"inferSchema is not supported for hive data source.") | ||
} | ||
|
||
override def prepareWrite( | ||
sparkSession: SparkSession, | ||
job: Job, | ||
options: Map[String, String], | ||
dataSchema: StructType): OutputWriterFactory = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Want to comment the original source of code in this function? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The preparation logic was dispersive before, I collected all of them and put them here. |
||
val conf = job.getConfiguration | ||
val tableDesc = fileSinkConf.getTableInfo | ||
conf.set("mapred.output.format.class", tableDesc.getOutputFileFormatClassName) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
|
||
// When speculation is on and output committer class name contains "Direct", we should warn | ||
// users that they may loss data if they are using a direct output committer. | ||
val speculationEnabled = sparkSession.sparkContext.conf.getBoolean("spark.speculation", false) | ||
val outputCommitterClass = conf.get("mapred.output.committer.class", "") | ||
if (speculationEnabled && outputCommitterClass.contains("Direct")) { | ||
val warningMessage = | ||
s"$outputCommitterClass may be an output committer that writes data directly to " + | ||
"the final location. Because speculation is enabled, this output committer may " + | ||
"cause data loss (see the case in SPARK-10063). If possible, please use an output " + | ||
"committer that does not have this behavior (e.g. FileOutputCommitter)." | ||
logWarning(warningMessage) | ||
} | ||
|
||
// Add table properties from storage handler to hadoopConf, so any custom storage | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
// handler settings can be set to hadoopConf | ||
HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, conf, false) | ||
Utilities.copyTableJobPropertiesToConf(tableDesc, conf) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will tableDesc be null? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the |
||
|
||
// Avoid referencing the outer object. | ||
val fileSinkConfSer = fileSinkConf | ||
new OutputWriterFactory { | ||
private val jobConf = new SerializableJobConf(new JobConf(conf)) | ||
@transient private lazy val outputFormat = | ||
jobConf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]] | ||
|
||
override def getFileExtension(context: TaskAttemptContext): String = { | ||
Utilities.getFileExtension(jobConf.value, fileSinkConfSer.getCompressed, outputFormat) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
} | ||
|
||
override def newInstance( | ||
path: String, | ||
dataSchema: StructType, | ||
context: TaskAttemptContext): OutputWriter = { | ||
new HiveOutputWriter(path, fileSinkConfSer, jobConf.value, dataSchema) | ||
} | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should we just create a class instead of using an anonymous class? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I followed other |
||
} | ||
} | ||
|
||
class HiveOutputWriter( | ||
path: String, | ||
fileSinkConf: FileSinkDesc, | ||
jobConf: JobConf, | ||
dataSchema: StructType) extends OutputWriter with HiveInspectors { | ||
|
||
private def tableDesc = fileSinkConf.getTableInfo | ||
|
||
private val serializer = { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] | ||
serializer.initialize(null, tableDesc.getProperties) | ||
serializer | ||
} | ||
|
||
private val hiveWriter = HiveFileFormatUtils.getHiveRecordWriter( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
jobConf, | ||
tableDesc, | ||
serializer.getSerializedClass, | ||
fileSinkConf, | ||
new Path(path), | ||
Reporter.NULL) | ||
|
||
private val standardOI = ObjectInspectorUtils | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. code block below(until |
||
.getStandardObjectInspector( | ||
tableDesc.getDeserializer.getObjectInspector, | ||
ObjectInspectorCopyOption.JAVA) | ||
.asInstanceOf[StructObjectInspector] | ||
|
||
private val fieldOIs = | ||
standardOI.getAllStructFieldRefs.asScala.map(_.getFieldObjectInspector).toArray | ||
private val dataTypes = dataSchema.map(_.dataType).toArray | ||
private val wrappers = fieldOIs.zip(dataTypes).map { case (f, dt) => wrapperFor(f, dt) } | ||
private val outputData = new Array[Any](fieldOIs.length) | ||
|
||
override def write(row: InternalRow): Unit = { | ||
var i = 0 | ||
while (i < fieldOIs.length) { | ||
outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) | ||
i += 1 | ||
} | ||
hiveWriter.write(serializer.serialize(outputData, standardOI)) | ||
} | ||
|
||
override def close(): Unit = { | ||
// Seems the boolean value passed into close does not matter. | ||
hiveWriter.close(false) | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Explain the reason that
SQLExecution.withNewExecutionId(sparkSession, this)
is not needed?