|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.hive.execution |
| 19 | + |
| 20 | +import scala.collection.JavaConverters._ |
| 21 | + |
| 22 | +import org.apache.hadoop.fs.{FileStatus, Path} |
| 23 | +import org.apache.hadoop.hive.ql.exec.Utilities |
| 24 | +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} |
| 25 | +import org.apache.hadoop.hive.serde2.Serializer |
| 26 | +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, StructObjectInspector} |
| 27 | +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption |
| 28 | +import org.apache.hadoop.io.Writable |
| 29 | +import org.apache.hadoop.mapred.{JobConf, Reporter} |
| 30 | +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} |
| 31 | + |
| 32 | +import org.apache.spark.internal.Logging |
| 33 | +import org.apache.spark.sql.SparkSession |
| 34 | +import org.apache.spark.sql.catalyst.InternalRow |
| 35 | +import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory} |
| 36 | +import org.apache.spark.sql.hive.{HiveInspectors, HiveTableUtil} |
| 37 | +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} |
| 38 | +import org.apache.spark.sql.types.StructType |
| 39 | +import org.apache.spark.util.SerializableJobConf |
| 40 | + |
| 41 | +/** |
| 42 | + * `FileFormat` for writing Hive tables. |
| 43 | + * |
| 44 | + * TODO: implement the read logic. |
| 45 | + */ |
| 46 | +class HiveFileFormat(fileSinkConf: FileSinkDesc) extends FileFormat with Logging { |
| 47 | + override def inferSchema( |
| 48 | + sparkSession: SparkSession, |
| 49 | + options: Map[String, String], |
| 50 | + files: Seq[FileStatus]): Option[StructType] = { |
| 51 | + throw new UnsupportedOperationException(s"inferSchema is not supported for hive data source.") |
| 52 | + } |
| 53 | + |
| 54 | + override def prepareWrite( |
| 55 | + sparkSession: SparkSession, |
| 56 | + job: Job, |
| 57 | + options: Map[String, String], |
| 58 | + dataSchema: StructType): OutputWriterFactory = { |
| 59 | + val conf = job.getConfiguration |
| 60 | + val tableDesc = fileSinkConf.getTableInfo |
| 61 | + conf.set("mapred.output.format.class", tableDesc.getOutputFileFormatClassName) |
| 62 | + |
| 63 | + // When speculation is on and output committer class name contains "Direct", we should warn |
| 64 | + // users that they may loss data if they are using a direct output committer. |
| 65 | + val speculationEnabled = sparkSession.sparkContext.conf.getBoolean("spark.speculation", false) |
| 66 | + val outputCommitterClass = conf.get("mapred.output.committer.class", "") |
| 67 | + if (speculationEnabled && outputCommitterClass.contains("Direct")) { |
| 68 | + val warningMessage = |
| 69 | + s"$outputCommitterClass may be an output committer that writes data directly to " + |
| 70 | + "the final location. Because speculation is enabled, this output committer may " + |
| 71 | + "cause data loss (see the case in SPARK-10063). If possible, please use an output " + |
| 72 | + "committer that does not have this behavior (e.g. FileOutputCommitter)." |
| 73 | + logWarning(warningMessage) |
| 74 | + } |
| 75 | + |
| 76 | + // Add table properties from storage handler to hadoopConf, so any custom storage |
| 77 | + // handler settings can be set to hadoopConf |
| 78 | + HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, conf, false) |
| 79 | + Utilities.copyTableJobPropertiesToConf(tableDesc, conf) |
| 80 | + |
| 81 | + // Avoid referencing the outer object. |
| 82 | + val fileSinkConfSer = fileSinkConf |
| 83 | + new OutputWriterFactory { |
| 84 | + private val jobConf = new SerializableJobConf(new JobConf(conf)) |
| 85 | + @transient private lazy val outputFormat = |
| 86 | + jobConf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]] |
| 87 | + |
| 88 | + override def getFileExtension(context: TaskAttemptContext): String = { |
| 89 | + Utilities.getFileExtension(jobConf.value, fileSinkConfSer.getCompressed, outputFormat) |
| 90 | + } |
| 91 | + |
| 92 | + override def newInstance( |
| 93 | + path: String, |
| 94 | + dataSchema: StructType, |
| 95 | + context: TaskAttemptContext): OutputWriter = { |
| 96 | + new HiveOutputWriter(path, fileSinkConfSer, jobConf.value, dataSchema) |
| 97 | + } |
| 98 | + } |
| 99 | + } |
| 100 | +} |
| 101 | + |
| 102 | +class HiveOutputWriter( |
| 103 | + path: String, |
| 104 | + fileSinkConf: FileSinkDesc, |
| 105 | + jobConf: JobConf, |
| 106 | + dataSchema: StructType) extends OutputWriter with HiveInspectors { |
| 107 | + |
| 108 | + private def tableDesc = fileSinkConf.getTableInfo |
| 109 | + |
| 110 | + private val serializer = { |
| 111 | + val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] |
| 112 | + serializer.initialize(null, tableDesc.getProperties) |
| 113 | + serializer |
| 114 | + } |
| 115 | + |
| 116 | + private val hiveWriter = HiveFileFormatUtils.getHiveRecordWriter( |
| 117 | + jobConf, |
| 118 | + tableDesc, |
| 119 | + serializer.getSerializedClass, |
| 120 | + fileSinkConf, |
| 121 | + new Path(path), |
| 122 | + Reporter.NULL) |
| 123 | + |
| 124 | + private val standardOI = ObjectInspectorUtils |
| 125 | + .getStandardObjectInspector( |
| 126 | + tableDesc.getDeserializer.getObjectInspector, |
| 127 | + ObjectInspectorCopyOption.JAVA) |
| 128 | + .asInstanceOf[StructObjectInspector] |
| 129 | + |
| 130 | + private val fieldOIs = |
| 131 | + standardOI.getAllStructFieldRefs.asScala.map(_.getFieldObjectInspector).toArray |
| 132 | + private val dataTypes = dataSchema.map(_.dataType).toArray |
| 133 | + private val wrappers = fieldOIs.zip(dataTypes).map { case (f, dt) => wrapperFor(f, dt) } |
| 134 | + private val outputData = new Array[Any](fieldOIs.length) |
| 135 | + |
| 136 | + override def write(row: InternalRow): Unit = { |
| 137 | + var i = 0 |
| 138 | + while (i < fieldOIs.length) { |
| 139 | + outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) |
| 140 | + i += 1 |
| 141 | + } |
| 142 | + hiveWriter.write(serializer.serialize(outputData, standardOI)) |
| 143 | + } |
| 144 | + |
| 145 | + override def close(): Unit = { |
| 146 | + // Seems the boolean value passed into close does not matter. |
| 147 | + hiveWriter.close(false) |
| 148 | + } |
| 149 | +} |
0 commit comments