From 59e99f6d7aad7c464c68fbb6e568f9347f6774ac Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 21 Apr 2020 21:31:07 -0700 Subject: [PATCH] Use taskAttemptId. --- .../scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala index b29b56b351f0..576a83f6ab4d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala @@ -199,8 +199,7 @@ private[spark] object ReliableCheckpointRDD extends Logging { val finalOutputName = ReliableCheckpointRDD.checkpointFileName(ctx.partitionId()) val finalOutputPath = new Path(outputDir, finalOutputName) - val tempOutputPath = new Path(outputDir, - s".$finalOutputName-attempt-${ctx.stageAttemptNumber()}-${ctx.attemptNumber()}") + val tempOutputPath = new Path(outputDir, s".$finalOutputName-attempt-${ctx.taskAttemptId()}") val bufferSize = env.conf.get(BUFFER_SIZE)