This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
     new 63fa6f5  [SPARK-26682][SQL] Use taskAttemptID instead of attemptNumber 
for Hadoop.
63fa6f5 is described below

commit 63fa6f5abc0c529d017243a4eea505c1c4cbbbd4
Author: Ryan Blue <b...@apache.org>
AuthorDate: Thu Jan 24 12:45:25 2019 +0800

    [SPARK-26682][SQL] Use taskAttemptID instead of attemptNumber for Hadoop.
    
    ## What changes were proposed in this pull request?
    
    Updates the attempt ID used by FileFormatWriter. Tasks in stage attempts 
use the same task attempt number and could conflict. Using Spark's task attempt 
ID guarantees that Hadoop TaskAttemptID instances are unique.
    
    ## How was this patch tested?
    
    Existing tests. Also validated that we no longer detect this failure case 
in our logs after deployment.
    
    Closes #23608 from rdblue/SPARK-26682-fix-hadoop-task-attempt-id.
    
    Authored-by: Ryan Blue <b...@apache.org>
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
    (cherry picked from commit d5a97c1c2c86ae335e91008fa25b3359c4560915)
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
---
 .../org/apache/spark/sql/execution/datasources/FileFormatWriter.scala   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

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 774fe38..2103a2d 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
@@ -170,7 +170,7 @@ object FileFormatWriter extends Logging {
             description = description,
             sparkStageId = taskContext.stageId(),
             sparkPartitionId = taskContext.partitionId(),
-            sparkAttemptNumber = taskContext.attemptNumber(),
+            sparkAttemptNumber = taskContext.taskAttemptId().toInt & 
Integer.MAX_VALUE,
             committer,
             iterator = iter)
         },


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to