From 56c5ff5b5a11a0ef7d5f4055cbf67bbbc310e111 Mon Sep 17 00:00:00 2001 From: s71955 Date: Thu, 27 Sep 2018 21:03:51 +0530 Subject: [PATCH] [SPARK-25521][SQL]Job id showing null in the logs when insert into command Job is finished. ## What changes were proposed in this pull request? As part of insert command in FileFormatWriter, a job context is created for handling the write operation , While initializing the job context setupJob() API in HadoopMapReduceCommitProtocol sets the jobid in the Jobcontext configuration, Since we are directly getting the jobId from the map reduce JobContext the job id will come as null in the logs. As a solution we shall get the jobID from the configuration of the map reduce Jobcontext ## How was this patch tested? Manually, verified the logs after the changes. --- .../spark/sql/execution/datasources/FileFormatWriter.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 7c6ab4bc922f..774fe38f5c2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -183,15 +183,15 @@ object FileFormatWriter extends Logging { val commitMsgs = ret.map(_.commitMsg) committer.commitJob(job, commitMsgs) - logInfo(s"Job ${job.getJobID} committed.") + logInfo(s"Write Job ${description.uuid} committed.") processStats(description.statsTrackers, ret.map(_.summary.stats)) - logInfo(s"Finished processing stats for job ${job.getJobID}.") + logInfo(s"Finished processing stats for write job ${description.uuid}.") // return a set of all the partition paths that were updated during this job ret.map(_.summary.updatedPartitions).reduceOption(_ ++ _).getOrElse(Set.empty) } catch { case cause: Throwable => - logError(s"Aborting job ${job.getJobID}.", cause) + logError(s"Aborting job ${description.uuid}.", cause) committer.abortJob(job) throw new SparkException("Job aborted.", cause) }