Skip to content
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

Closed
wants to merge 3 commits into from
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.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -99,7 +99,7 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
}

private def getFilename(taskContext: TaskAttemptContext, ext: String): String = {
// The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet
// The file name looks like part-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003-c000.parquet
// Note that %05d does not truncate the split number, so if we have more than 100000 tasks,
// the file name is fine and won't overflow.
val split = taskContext.getTaskAttemptID.getTaskID.getId
Expand Down
Expand Up @@ -108,35 +108,30 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) {


/**
* Returns the result as a hive compatible sequence of strings. For native commands, the
* execution is simply passed back to Hive.
* Returns the result as a hive compatible sequence of strings. This is for testing only.
*/
def hiveResultString(): Seq[String] = executedPlan match {
case ExecutedCommandExec(desc: DescribeTableCommand) =>
SQLExecution.withNewExecutionId(sparkSession, this) {
// If it is a describe command for a Hive table, we want to have the output format
// be similar with Hive.
desc.run(sparkSession).map {
case Row(name: String, dataType: String, comment) =>
Seq(name, dataType,
Option(comment.asInstanceOf[String]).getOrElse(""))
.map(s => String.format(s"%-20s", s))
.mkString("\t")
}
// If it is a describe command for a Hive table, we want to have the output format
// be similar with Hive.
Copy link
Contributor

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?

desc.run(sparkSession).map {
case Row(name: String, dataType: String, comment) =>
Seq(name, dataType,
Option(comment.asInstanceOf[String]).getOrElse(""))
.map(s => String.format(s"%-20s", s))
.mkString("\t")
}
// SHOW TABLES in Hive only output table names, while ours outputs database, table name, isTemp.
case command: ExecutedCommandExec if command.cmd.isInstanceOf[ShowTablesCommand] =>
command.executeCollect().map(_.getString(1))
case command: ExecutedCommandExec =>
command.executeCollect().map(_.getString(0))
case other =>
SQLExecution.withNewExecutionId(sparkSession, this) {
val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq
// We need the types so we can output struct field names
val types = analyzed.output.map(_.dataType)
// Reformat to match hive tab delimited output.
result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq
}
val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq
// We need the types so we can output struct field names
val types = analyzed.output.map(_.dataType)
// Reformat to match hive tab delimited output.
result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t"))
}

/** Formats a datum (based on the given data type) and returns the string representation. */
Expand Down
Expand Up @@ -66,6 +66,7 @@ private[hive] class HiveSessionState(sparkSession: SparkSession)
PreprocessTableInsertion(conf) ::
DataSourceAnalysis(conf) ::
new DetermineHiveSerde(conf) ::
new HiveAnalysis(sparkSession) ::
new ResolveDataSource(sparkSession) :: Nil

override val extendedCheckRules = Seq(PreWriteCheck(conf, catalog))
Expand All @@ -88,7 +89,6 @@ private[hive] class HiveSessionState(sparkSession: SparkSession)
SpecialLimits,
InMemoryScans,
HiveTableScans,
DataSinks,
Scripts,
Aggregation,
JoinSelection,
Expand Down
Expand Up @@ -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


/**
Expand Down Expand Up @@ -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)
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

to use FileFormatWriter, we need to make InsertIntoHiveTable a LogicalPlan like InsertIntoHadoopFsRelation, which means we need to convert InsertIntoTable to InsertIntoHiveTable during analysis.


case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) =>
// Currently `DataFrameWriter.saveAsTable` doesn't support the Append mode of hive serde
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

// 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
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

Copy link
Contributor

Choose a reason for hiding this comment

The 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?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this function actually be part of the resolved method of InsertIntoTable?

Copy link
Contributor

Choose a reason for hiding this comment

The 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 =>
Expand All @@ -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
}
}
Expand Down
Expand Up @@ -311,10 +311,10 @@ private[hive] object HiveTableUtil {
// that calls Hive.get() which tries to access metastore, but it's not valid in runtime
// it would be fixed in next version of hive but till then, we should use this instead
def configureJobPropertiesForStorageHandler(
tableDesc: TableDesc, jobConf: JobConf, input: Boolean) {
tableDesc: TableDesc, conf: Configuration, input: Boolean) {
val property = tableDesc.getProperties.getProperty(META_TABLE_STORAGE)
val storageHandler =
org.apache.hadoop.hive.ql.metadata.HiveUtils.getStorageHandler(jobConf, property)
org.apache.hadoop.hive.ql.metadata.HiveUtils.getStorageHandler(conf, property)
if (storageHandler != null) {
val jobProperties = new java.util.LinkedHashMap[String, String]
if (input) {
Expand Down
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive

/** Support for interacting with different versions of the HiveMetastoreClient */
package object client {
private[client] abstract class HiveVersion(
private[hive] abstract class HiveVersion(
val fullVersion: String,
val extraDeps: Seq[String] = Nil,
val exclusions: Seq[String] = Nil)
Expand Down
@@ -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 = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Want to comment the original source of code in this function?

Copy link
Contributor Author

@cloud-fan cloud-fan Jan 17, 2017

Choose a reason for hiding this comment

The 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)
Copy link
Contributor Author

Choose a reason for hiding this comment

The 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
Copy link
Contributor Author

Choose a reason for hiding this comment

The 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)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will tableDesc be null?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the tableDesc is created at https://github.com/apache/spark/pull/16517/files#diff-d579db9a8f27e0bbef37720ab14ec3f6R223 . So it will never be null, and the previous null check is unnecessary.


// 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)
Copy link
Contributor Author

Choose a reason for hiding this comment

The 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)
}
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we just create a class instead of using an anonymous class?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I followed other FileFormat implementations here.

}
}

class HiveOutputWriter(
path: String,
fileSinkConf: FileSinkDesc,
jobConf: JobConf,
dataSchema: StructType) extends OutputWriter with HiveInspectors {

private def tableDesc = fileSinkConf.getTableInfo

private val serializer = {
Copy link
Contributor Author

Choose a reason for hiding this comment

The 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(
Copy link
Contributor Author

Choose a reason for hiding this comment

The 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
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

.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)
}
}