Skip to content

Commit d471b74

Browse files
author
Marcelo Vanzin
committed
Fix sql compilation.
1 parent 09e5d15 commit d471b74

File tree

1 file changed

+6
-3
lines changed

1 file changed

+6
-3
lines changed

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

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -109,6 +109,7 @@ object DataWritingSparkTask extends Logging {
109109
iter: Iterator[InternalRow],
110110
useCommitCoordinator: Boolean): WriterCommitMessage = {
111111
val stageId = context.stageId()
112+
val stageAttempt = context.stageAttemptNumber()
112113
val partId = context.partitionId()
113114
val attemptId = context.attemptNumber()
114115
val epochId = Option(context.getLocalProperty(MicroBatchExecution.BATCH_ID_KEY)).getOrElse("0")
@@ -122,12 +123,14 @@ object DataWritingSparkTask extends Logging {
122123

123124
val msg = if (useCommitCoordinator) {
124125
val coordinator = SparkEnv.get.outputCommitCoordinator
125-
val commitAuthorized = coordinator.canCommit(context.stageId(), partId, attemptId)
126+
val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
126127
if (commitAuthorized) {
127-
logInfo(s"Writer for stage $stageId, task $partId.$attemptId is authorized to commit.")
128+
logInfo(s"Writer for stage $stageId / $stageAttempt, " +
129+
s"task $partId.$attemptId is authorized to commit.")
128130
dataWriter.commit()
129131
} else {
130-
val message = s"Stage $stageId, task $partId.$attemptId: driver did not authorize commit"
132+
val message = s"Stage $stageId / $stageAttempt, " +
133+
s"task $partId.$attemptId: driver did not authorize commit"
131134
logInfo(message)
132135
// throwing CommitDeniedException will trigger the catch block for abort
133136
throw new CommitDeniedException(message, stageId, partId, attemptId)

0 commit comments

Comments
 (0)