Skip to content

[SPARK-41407][SQL][FOLLOW-UP] Use string jobTrackerID for FileFormatWriter.executeTask #39194

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 1 commit into from
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources
import java.util.Date

import org.apache.spark.{SparkException, TaskContext}
import org.apache.spark.internal.io.FileCommitProtocol
import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
Expand Down Expand Up @@ -72,15 +72,15 @@ case class WriteFilesExec(child: SparkPlan) extends UnaryExecNode {
val concurrentOutputWriterSpec = writeFilesSpec.concurrentOutputWriterSpecFunc(child)
val description = writeFilesSpec.description
val committer = writeFilesSpec.committer
val jobIdInstant = new Date().getTime
val jobTrackerID = SparkHadoopWriterUtils.createJobTrackerID(new Date())
rddWithNonEmptyPartitions.mapPartitionsInternal { iterator =>
val sparkStageId = TaskContext.get().stageId()
val sparkPartitionId = TaskContext.get().partitionId()
val sparkAttemptNumber = TaskContext.get().taskAttemptId().toInt & Int.MaxValue

val ret = FileFormatWriter.executeTask(
description,
jobIdInstant,
jobTrackerID,
sparkStageId,
sparkPartitionId,
sparkAttemptNumber,
Expand Down