Skip to content

Commit 4597007

Browse files
sujith71955cloud-fan
authored andcommitted
[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 using setupJob() API in HadoopMapReduceCommitProtocol , we set the jobid in the Jobcontext configuration.In FileFormatWriter since we are directly getting the jobId from the map reduce JobContext the job id will come as null while adding the log. 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-25521 1](https://user-images.githubusercontent.com/12999161/46164933-e95ab700-c2ac-11e8-88e9-49fa5100b872.PNG) Closes apache#22572 from sujith71955/master_log_issue. Authored-by: s71955 <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
1 parent 5ae20cf commit 4597007

File tree

1 file changed

+3
-3
lines changed

1 file changed

+3
-3
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -183,15 +183,15 @@ object FileFormatWriter extends Logging {
183183
val commitMsgs = ret.map(_.commitMsg)
184184

185185
committer.commitJob(job, commitMsgs)
186-
logInfo(s"Job ${job.getJobID} committed.")
186+
logInfo(s"Write Job ${description.uuid} committed.")
187187

188188
processStats(description.statsTrackers, ret.map(_.summary.stats))
189-
logInfo(s"Finished processing stats for job ${job.getJobID}.")
189+
logInfo(s"Finished processing stats for write job ${description.uuid}.")
190190

191191
// return a set of all the partition paths that were updated during this job
192192
ret.map(_.summary.updatedPartitions).reduceOption(_ ++ _).getOrElse(Set.empty)
193193
} catch { case cause: Throwable =>
194-
logError(s"Aborting job ${job.getJobID}.", cause)
194+
logError(s"Aborting job ${description.uuid}.", cause)
195195
committer.abortJob(job)
196196
throw new SparkException("Job aborted.", cause)
197197
}

0 commit comments

Comments
 (0)