-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23559][SS] Add epoch ID to DataWriterFactory. #20710
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 6 commits
42bca60
84cfa21
a18a57b
b2ee7f3
55b38db
544eb1b
f5948e8
4588616
79495b1
9fb74e2
215c225
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 |
|---|---|---|
|
|
@@ -48,6 +48,9 @@ public interface DataWriterFactory<T> extends Serializable { | |
| * same task id but different attempt number, which means there are multiple | ||
| * tasks with the same task id running at the same time. Implementations can | ||
| * use this attempt number to distinguish writers of different task attempts. | ||
| * @param epochId A monotonically increasing id for streaming queries that are split in to | ||
| * discrete periods of execution. For queries that execute as a single batch, this | ||
|
||
| * id will always be zero. | ||
| */ | ||
| DataWriter<T> createDataWriter(int partitionId, int attemptNumber); | ||
| DataWriter<T> createDataWriter(int partitionId, int attemptNumber, long epochId); | ||
|
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. Add clear lifecycle semantics.
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. Why are we using the same interface for streaming and batch here? Is there a compelling reason to do so instead of adding
Contributor
Author
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. The guarantees are identical, and in the current execution model, each epoch is in fact processed by a single batch job. |
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,10 +24,7 @@ | |
|
|
||
| /** | ||
| * A {@link DataSourceWriter} for use with structured streaming. This writer handles commits and | ||
| * aborts relative to an epoch ID determined by the execution engine. | ||
| * | ||
| * {@link DataWriter} implementations generated by a StreamWriter may be reused for multiple epochs, | ||
| * and so must reset any internal state after a successful commit. | ||
| * aborts relative to an epoch ID provided by the execution engine. | ||
| */ | ||
| @InterfaceStability.Evolving | ||
| public interface StreamWriter extends DataSourceWriter { | ||
|
|
@@ -39,21 +36,21 @@ public interface StreamWriter extends DataSourceWriter { | |
| * If this method fails (by throwing an exception), this writing job is considered to have been | ||
| * failed, and the execution engine will attempt to call {@link #abort(WriterCommitMessage[])}. | ||
| * | ||
| * To support exactly-once processing, writer implementations should ensure that this method is | ||
| * idempotent. The execution engine may call commit() multiple times for the same epoch | ||
| * in some circumstances. | ||
| * The execution engine may call commit() multiple times for the same epoch in some circumstances. | ||
|
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. Somewhere in this file, add docs about what epochId means for MB and C execution.
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. +1 |
||
| * To support exactly-once data semantics, implementations must ensure that multiple commits for | ||
| * the same epoch are idempotent. | ||
| */ | ||
| void commit(long epochId, WriterCommitMessage[] messages); | ||
|
|
||
| /** | ||
| * Aborts this writing job because some data writers are failed and keep failing when retry, or | ||
| * Aborts this writing job because some data writers are failed and keep failing when retried, or | ||
| * the Spark job fails with some unknown reasons, or {@link #commit(WriterCommitMessage[])} fails. | ||
| * | ||
| * If this method fails (by throwing an exception), the underlying data source may require manual | ||
| * cleanup. | ||
| * | ||
| * Unless the abort is triggered by the failure of commit, the given messages should have some | ||
| * null slots as there maybe only a few data writers that are committed before the abort | ||
| * Unless the abort is triggered by the failure of commit, the given messages will have some | ||
| * null slots, as there may be only a few data writers that were committed before the abort | ||
| * happens, or some data writers were committed but their commit messages haven't reached the | ||
| * driver when the abort is triggered. So this is just a "best effort" for data sources to | ||
| * clean up the data left by data writers. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,7 +29,7 @@ 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.StreamExecution | ||
| 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.sources.v2.writer._ | ||
| import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter | ||
|
|
@@ -132,7 +132,8 @@ object DataWritingSparkTask extends Logging { | |
| val stageId = context.stageId() | ||
| val partId = context.partitionId() | ||
| val attemptId = context.attemptNumber() | ||
| val dataWriter = writeTask.createDataWriter(partId, attemptId) | ||
| val epochId = Option(context.getLocalProperty(MicroBatchExecution.BATCH_ID_KEY)).getOrElse("0") | ||
| val dataWriter = writeTask.createDataWriter(partId, attemptId, epochId.toLong) | ||
|
|
||
| // write the data and commit this writer. | ||
| Utils.tryWithSafeFinallyAndFailureCallbacks(block = { | ||
|
|
@@ -172,17 +173,19 @@ object DataWritingSparkTask extends Logging { | |
| writeTask: DataWriterFactory[InternalRow], | ||
| context: TaskContext, | ||
| iter: Iterator[InternalRow]): WriterCommitMessage = { | ||
| val dataWriter = writeTask.createDataWriter(context.partitionId(), context.attemptNumber()) | ||
| val epochCoordinator = EpochCoordinatorRef.get( | ||
| context.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY), | ||
| SparkEnv.get) | ||
| val currentMsg: WriterCommitMessage = null | ||
| var currentEpoch = context.getLocalProperty(ContinuousExecution.START_EPOCH_KEY).toLong | ||
|
|
||
| do { | ||
| var dataWriter: DataWriter[InternalRow] = null | ||
| // write the data and commit this writer. | ||
| Utils.tryWithSafeFinallyAndFailureCallbacks(block = { | ||
| try { | ||
| dataWriter = writeTask.createDataWriter( | ||
| context.partitionId(), context.attemptNumber(), currentEpoch) | ||
| iter.foreach(dataWriter.write) | ||
|
||
| logInfo(s"Writer for partition ${context.partitionId()} is committing.") | ||
| val msg = dataWriter.commit() | ||
|
|
@@ -198,7 +201,7 @@ object DataWritingSparkTask extends Logging { | |
| })(catchBlock = { | ||
| // If there is an error, abort this writer | ||
| logError(s"Writer for partition ${context.partitionId()} is aborting.") | ||
| dataWriter.abort() | ||
| if (dataWriter != null) dataWriter.abort() | ||
| logError(s"Writer for partition ${context.partitionId()} aborted.") | ||
|
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. nit: add comment that the exception will be rethrown. |
||
| }) | ||
| } while (!context.isInterrupted()) | ||
|
|
@@ -211,9 +214,12 @@ class InternalRowDataWriterFactory( | |
| rowWriterFactory: DataWriterFactory[Row], | ||
| schema: StructType) extends DataWriterFactory[InternalRow] { | ||
|
|
||
| override def createDataWriter(partitionId: Int, attemptNumber: Int): DataWriter[InternalRow] = { | ||
| override def createDataWriter( | ||
| partitionId: Int, | ||
| attemptNumber: Int, | ||
| epochId: Long): DataWriter[InternalRow] = { | ||
| new InternalRowDataWriter( | ||
| rowWriterFactory.createDataWriter(partitionId, attemptNumber), | ||
| rowWriterFactory.createDataWriter(partitionId, attemptNumber, 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.
Spark may retry... (in continuous we dont retry the task)
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.
for some times --> for a few times
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.
Break this sentence. very long.