Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16517#discussion_r96331505
  
    --- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala
 ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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.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 {
    +  override def inferSchema(
    +      sparkSession: SparkSession,
    +      options: Map[String, String],
    +      files: Seq[FileStatus]): Option[StructType] = None
    +
    +  override def prepareWrite(
    +      sparkSession: SparkSession,
    +      job: Job,
    +      options: Map[String, String],
    +      dataSchema: StructType): OutputWriterFactory = {
    +    val conf = job.getConfiguration
    +    val tableDesc = fileSinkConf.getTableInfo
    +    conf.set("mapred.output.format.class", 
tableDesc.getOutputFileFormatClassName)
    +
    +    // Add table properties from storage handler to hadoopConf, so any 
custom storage
    +    // handler settings can be set to hadoopConf
    +    HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, conf, 
false)
    +    Utilities.copyTableJobPropertiesToConf(tableDesc, conf)
    +
    +    // 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)
    +      }
    +
    +      override def newInstance(
    +          path: String,
    +          dataSchema: StructType,
    +          context: TaskAttemptContext): OutputWriter = {
    +        new HiveOutputWriter(path, fileSinkConfSer, jobConf.value, 
dataSchema)
    +      }
    +    }
    +  }
    +}
    +
    +class HiveOutputWriter(
    +    path: String,
    +    fileSinkConf: FileSinkDesc,
    +    jobConf: JobConf,
    +    dataSchema: StructType) extends OutputWriter with HiveInspectors {
    +
    +  private def tableDesc = fileSinkConf.getTableInfo
    +
    +  private val serializer = {
    +    val serializer = 
tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer]
    +    serializer.initialize(null, tableDesc.getProperties)
    +    serializer
    +  }
    +
    +  private val hiveWriter = HiveFileFormatUtils.getHiveRecordWriter(
    --- End diff --
    
    this is moved from 
https://github.com/apache/spark/pull/16517/files#diff-92b05808926122b334c2fdd2fd1e4221L121


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to