[GitHub] spark pull request #21558: [SPARK-24552][SQL] Use task ID instead of attempt...

2018-06-25 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/21558


---

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



[GitHub] spark pull request #21558: [SPARK-24552][SQL] Use task ID instead of attempt...

2018-06-21 Thread tgravescs
Github user tgravescs commented on a diff in the pull request:

https://github.com/apache/spark/pull/21558#discussion_r197222759
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
 ---
@@ -110,7 +108,7 @@ object DataWritingSparkTask extends Logging {
   useCommitCoordinator: Boolean): WriterCommitMessage = {
 val stageId = context.stageId()
 val partId = context.partitionId()
-val attemptId = context.attemptNumber()
+val attemptId = context.taskAttemptId().toInt
--- End diff --

HadoopWriteConfigUtil has the same issue, its a public interface and uses 
in for attempt number. 
it seems somewhat unlikely but more likely to be able to go over an int for 
task ids in spark then in say MapReduce.  we do have partitionId as an Int so 
if partitions go to Int and you have task failures then taskids could go over 
Int.  Looking at our options




---

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



[GitHub] spark pull request #21558: [SPARK-24552][SQL] Use task ID instead of attempt...

2018-06-19 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/21558#discussion_r196592175
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
 ---
@@ -110,7 +108,7 @@ object DataWritingSparkTask extends Logging {
   useCommitCoordinator: Boolean): WriterCommitMessage = {
 val stageId = context.stageId()
 val partId = context.partitionId()
-val attemptId = context.attemptNumber()
+val attemptId = context.taskAttemptId().toInt
--- End diff --

I was going to suggest removing the cast to int, but well, that's in 
`DataWriterFactory` and would be an API breakage... hopefully won't cause 
issues aside from weird output names when the value overflows the int.


---

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



[GitHub] spark pull request #21558: [SPARK-24552][SQL] Use task ID instead of attempt...

2018-06-18 Thread squito
Github user squito commented on a diff in the pull request:

https://github.com/apache/spark/pull/21558#discussion_r196277478
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2.scala
 ---
@@ -110,7 +108,7 @@ object DataWritingSparkTask extends Logging {
   useCommitCoordinator: Boolean): WriterCommitMessage = {
 val stageId = context.stageId()
 val partId = context.partitionId()
-val attemptId = context.attemptNumber()
+val attemptId = context.taskAttemptId().toInt
--- End diff --

nit: can you rename this variable `tid`?  these names are pretty confusing, 
but I think that at least "tid" is used consistently and exclusively for this, 
while "attempt" means a lot of different things.


---

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



[GitHub] spark pull request #21558: [SPARK-24552][SQL] Use task ID instead of attempt...

2018-06-13 Thread rdblue
GitHub user rdblue opened a pull request:

https://github.com/apache/spark/pull/21558

[SPARK-24552][SQL] Use task ID instead of attempt number for v2 writes.

## What changes were proposed in this pull request?

This passes the unique task attempt id instead of attempt number to v2 data 
sources because attempt number is reused when stages are retried. When attempt 
numbers are reused, sources that track data by partition id and attempt number 
may incorrectly clean up data because **the same attempt number can be both 
committed and aborted**.

## How was this patch tested?

Existing tests.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/rdblue/spark SPARK-24552-v2-source-work-around

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/21558.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #21558


commit e9e776a097f5dca1dccdd6e50b3790e6a91873d8
Author: Ryan Blue 
Date:   2018-06-13T19:50:00Z

SPARK-24552: Use task ID instead of attempt number for v2 writes.




---

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