@@ -23,17 +23,21 @@ import java.security.PrivilegedExceptionAction
2323import java .text .SimpleDateFormat
2424import java .util .{Date , Locale , Random }
2525
26+ import scala .collection .JavaConverters ._
2627import scala .util .control .NonFatal
2728
2829import org .apache .hadoop .conf .Configuration
2930import org .apache .hadoop .fs .{FileSystem , Path }
3031import org .apache .hadoop .hdfs .DistributedFileSystem
3132import org .apache .hadoop .hive .common .FileUtils
32- import org .apache .hadoop .hive .ql .exec .TaskRunner
3333import org .apache .hadoop .hive .ql .ErrorMsg
34+ import org .apache .hadoop .hive .ql .exec .{TaskRunner , Utilities }
35+ import org .apache .hadoop .hive .ql .io .HiveOutputFormat
36+ import org .apache .hadoop .io .Writable
3437import org .apache .hadoop .mapred .{FileOutputFormat , JobConf }
3538import org .apache .hadoop .security .UserGroupInformation
3639
40+ import org .apache .spark .SparkException
3741import org .apache .spark .rdd .RDD
3842import org .apache .spark .sql .AnalysisException
3943import org .apache .spark .sql .catalyst .InternalRow
@@ -42,8 +46,9 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning
4246import org .apache .spark .sql .execution .{SparkPlan , UnaryExecNode }
4347import org .apache .spark .sql .hive ._
4448import org .apache .spark .sql .hive .HiveShim .{ShimFileSinkDesc => FileSinkDesc }
45- import org .apache .spark .SparkException
46- import org .apache .spark .util .SerializableJobConf
49+ import org .apache .spark .sql .hive .merge .MergeUtils
50+ import org .apache .spark .sql .internal .SQLConf
51+ import org .apache .spark .util .{RpcUtils , SerializableJobConf }
4752
4853
4954/**
@@ -94,6 +99,14 @@ case class InsertIntoHiveTable(
9499 val stagingDir = hadoopConf.get(" hive.exec.stagingdir" , " .hive-staging" )
95100 val scratchDir = hadoopConf.get(" hive.exec.scratchdir" , " /tmp/hive" )
96101
102+ private val avgConditionSize = sqlContext.sparkSession.conf
103+ .get(SQLConf .MERGE_SMALLFILE_SIZE )
104+ private val outputAverageSize = sqlContext.sparkSession
105+ .conf.get(SQLConf .MERGE_FILE_PER_TASK )
106+ private val mergeHiveFiles = sqlContext.sparkSession.sessionState.conf.mergeHiveFiles
107+ private val targetFileSize = Math .max(avgConditionSize, outputAverageSize)
108+ private val retryWaitMs = RpcUtils .retryWaitMs(sqlContext.sparkContext.conf)
109+
97110 private def executionId : String = {
98111 val rand : Random = new Random
99112 val format = new SimpleDateFormat (" yyyy-MM-dd_HH-mm-ss_SSS" , Locale .US )
@@ -222,6 +235,69 @@ case class InsertIntoHiveTable(
222235 new Path (getStagingDir(path), " -ext-10000" ) // Hive uses 10000
223236 }
224237
238+ private def mergeFile (
239+ path : Path ,
240+ fs : FileSystem ,
241+ fileSinkConf : FileSinkDesc ,
242+ conf : SerializableJobConf ,
243+ directRenamePathList : java.util.List [String ],
244+ speculationEnabled : Boolean ): Unit = {
245+ val hiveOutputFormat = conf.value.getOutputFormat
246+ .asInstanceOf [HiveOutputFormat [AnyRef , Writable ]]
247+ val extension = Utilities .getFileExtension(conf.value,
248+ fileSinkConf.getCompressed, hiveOutputFormat)
249+ val outputClassName = fileSinkConf.getTableInfo.getOutputFileFormatClassName
250+ val outputDir = path.toString
251+ val tmpMergeLocation = MergeUtils .getExternalMergeTmpPath(path, conf.value)
252+ val tmpMergeLocationDir = tmpMergeLocation.toString
253+ fileSinkConf.dir = tmpMergeLocation.toString
254+ val waitTime = retryWaitMs
255+ val numDynamicPartitions = partition.values.count(_.isEmpty)
256+ if (numDynamicPartitions > 0 ) {
257+ val mergeRules = MergeUtils .generateDynamicMergeRule(fs, path,
258+ conf.value, avgConditionSize, targetFileSize, directRenamePathList)
259+ sparkContext.union(mergeRules.map { r =>
260+ val groupSize = Math .ceil(r.files.size * 1d / r.numFiles).toInt
261+ val groupedFiles = r.files.toArray.grouped(groupSize).map(x => (r.path, x)).toArray
262+ MergeUtils .mergePathRDD(sparkContext, groupedFiles, groupedFiles.size)
263+ }).foreach { case (partOutputDir, files) =>
264+ val tmpPartMergeLocationDir = partOutputDir.replace(" -ext-10000" , MergeUtils .TEMP_DIR )
265+ MergeUtils .mergeAction(conf, outputClassName, files, partOutputDir, tmpPartMergeLocationDir,
266+ extension, waitTime)
267+ }
268+ if (speculationEnabled) {
269+ mergeRules.foreach { r =>
270+ val specFiles = fs.listStatus(
271+ new Path (r.path.toString.replace(" -ext-10000" , MergeUtils .TEMP_DIR )))
272+ .filter(! _.getPath.getName.startsWith(" part" ))
273+ specFiles.foreach(f => fs.delete(f.getPath))
274+ }
275+ }
276+ } else {
277+ val numFiles = MergeUtils .getTargetFileNum(path, conf.value,
278+ avgConditionSize, targetFileSize)
279+ if (numFiles > 0 ) {
280+ val files = fs.listStatus(path).filter(_.getLen > 0 ).map(_.getPath.toString)
281+ val groupSize = Math .ceil(files.size * 1d / numFiles).toInt
282+ val groupedFiles = files.grouped(groupSize).toArray
283+ fileSinkConf.dir = tmpMergeLocation.toString
284+ sparkContext.parallelize(groupedFiles, groupedFiles.size).foreach { files =>
285+ MergeUtils .mergeAction(conf, outputClassName, files, outputDir, tmpMergeLocationDir,
286+ extension, waitTime)
287+ }
288+ if (speculationEnabled) {
289+ val specFiles = fs.listStatus(tmpMergeLocation)
290+ .filter(! _.getPath.getName.startsWith(" part" ))
291+ specFiles.foreach(f => fs.delete(f.getPath))
292+ }
293+ if (conf.value.getBoolean(" mapreduce.fileoutputcommitter.marksuccessfuljobs" , true )) {
294+ fs.createNewFile(new Path (tmpMergeLocationDir + " /_SUCCESS" ))
295+ }
296+ }
297+ }
298+ FileOutputFormat .setOutputPath(conf.value, tmpMergeLocation)
299+ }
300+
225301 private def saveAsHiveFile (
226302 rdd : RDD [InternalRow ],
227303 valueClass : Class [_],
@@ -310,6 +386,7 @@ case class InsertIntoHiveTable(
310386 }
311387
312388 val jobConf = new JobConf (hadoopConf)
389+ jobConf.set(MergeUtils .SCHEMA , table.attributes.toStructType.json)
313390 val jobConfSer = new SerializableJobConf (jobConf)
314391
315392 // When speculation is on and output committer class name contains "Direct", we should warn
@@ -342,6 +419,38 @@ case class InsertIntoHiveTable(
342419 @ transient val outputClass = writerContainer.newSerializer(table.tableDesc).getSerializedClass
343420 saveAsHiveFile(child.execute(), outputClass, fileSinkConf, jobConfSer, writerContainer)
344421
422+ val outputFormatClass = fileSinkConf.getTableInfo.getOutputFileFormatClassName
423+ if (mergeHiveFiles && targetFileSize > 0 && ! table.databaseName.contains(" temp" ) &&
424+ MergeUtils .SUPPORTED_FORMAT .contains(outputFormatClass)) {
425+ val directRenamePathList = new java.util.ArrayList [String ]()
426+ val rollbackPathList = new java.util.ArrayList [String ]()
427+ val fs = tmpLocation.getFileSystem(jobConf)
428+ try {
429+ mergeFile(tmpLocation, fs, fileSinkConf, jobConfSer,
430+ directRenamePathList, speculationEnabled)
431+ if (! directRenamePathList.isEmpty) {
432+ directRenamePathList.asScala.foreach { path =>
433+ val destPath = path.replace(" -ext-10000" , MergeUtils .TEMP_DIR )
434+ rollbackPathList.add(path)
435+ logInfo(" rename [" + path + " to " + destPath + " ]" )
436+ fs.rename(new Path (path), new Path (destPath))
437+ }
438+ }
439+ } catch {
440+ case ex : Exception =>
441+ logInfo(" Merge file of " + tmpLocation + " failed!" , ex)
442+ fileSinkConf.dir = tmpLocation.toString
443+ if (! rollbackPathList.isEmpty) {
444+ rollbackPathList.asScala.foreach { path =>
445+ val srcPath = path.replace(" -ext-10000" , MergeUtils .TEMP_DIR )
446+ logInfo(" rename [" + srcPath + " to "
447+ + path + " ]" )
448+ fs.rename(new Path (srcPath), new Path (path))
449+ }
450+ }
451+ }
452+ }
453+
345454 val outputPath = FileOutputFormat .getOutputPath(jobConf)
346455 // TODO: Correctly set holdDDLTime.
347456 // In most of the time, we should have holdDDLTime = false.
0 commit comments