-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24552][core][SQL] Use task ID instead of attempt number for writes. #21606
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
Changes from 9 commits
6c60d14
2e65524
3561723
d5a079d
fdcd39c
7233a5f
c884f4f
5efaae7
227d513
a16d9f9
503852f
47131c5
e892937
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,10 +29,8 @@ import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} | |
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.execution.SparkPlan | ||
| import org.apache.spark.sql.execution.streaming.{MicroBatchExecution, StreamExecution} | ||
| import org.apache.spark.sql.execution.streaming.continuous.{CommitPartitionEpoch, ContinuousExecution, EpochCoordinatorRef, SetWriterPartitions} | ||
| import org.apache.spark.sql.execution.streaming.MicroBatchExecution | ||
| import org.apache.spark.sql.sources.v2.writer._ | ||
| import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.util.Utils | ||
|
|
||
|
|
@@ -111,9 +109,10 @@ object DataWritingSparkTask extends Logging { | |
| val stageId = context.stageId() | ||
| val stageAttempt = context.stageAttemptNumber() | ||
| val partId = context.partitionId() | ||
| val taskId = context.taskAttemptId() | ||
| val attemptId = context.attemptNumber() | ||
| val epochId = Option(context.getLocalProperty(MicroBatchExecution.BATCH_ID_KEY)).getOrElse("0") | ||
| val dataWriter = writeTask.createDataWriter(partId, attemptId, epochId.toLong) | ||
| val dataWriter = writeTask.createDataWriter(partId, taskId, epochId.toLong) | ||
|
|
||
| // write the data and commit this writer. | ||
| Utils.tryWithSafeFinallyAndFailureCallbacks(block = { | ||
|
|
@@ -125,11 +124,11 @@ object DataWritingSparkTask extends Logging { | |
| val coordinator = SparkEnv.get.outputCommitCoordinator | ||
| val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. a note for the followup: since we decided to use |
||
| if (commitAuthorized) { | ||
| logInfo(s"Writer for stage $stageId / $stageAttempt, " + | ||
| logInfo(s"Writer for stage $stageId.$stageAttempt, " + | ||
| s"task $partId.$attemptId is authorized to commit.") | ||
| dataWriter.commit() | ||
| } else { | ||
| val message = s"Stage $stageId / $stageAttempt, " + | ||
| val message = s"Stage $stageId.$stageAttempt, " + | ||
|
||
| s"task $partId.$attemptId: driver did not authorize commit" | ||
| logInfo(message) | ||
| // throwing CommitDeniedException will trigger the catch block for abort | ||
|
|
@@ -160,10 +159,10 @@ class InternalRowDataWriterFactory( | |
|
|
||
| override def createDataWriter( | ||
| partitionId: Int, | ||
| attemptNumber: Int, | ||
| taskId: Long, | ||
| epochId: Long): DataWriter[InternalRow] = { | ||
| new InternalRowDataWriter( | ||
| rowWriterFactory.createDataWriter(partitionId, attemptNumber, epochId), | ||
| rowWriterFactory.createDataWriter(partitionId, taskId, epochId), | ||
| RowEncoder.apply(schema).resolveAndBind()) | ||
| } | ||
| } | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
since it's not a simple
toIntanymore, how about we combine stage and task attempt number?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we can also remove the assert and assume that, even we have so many attempts, they are not all active.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok, I'll use that. I think Spark might fail everything before you even go that high in attempt numbers anyway...