-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19876][SS][WIP] OneTime Trigger Executor #17219
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 all commits
08a36f8
0e21d0e
9b8abb4
682eb1a
a129dd5
3e666b1
b4ef029
7cb43b7
98812cb
8c5b84f
573ec98
5d2ba62
2989fad
24746f3
fd28ed7
841f5c9
83db3b8
b02d10d
5e16632
d7c5edf
ae92ec6
64cd233
8b50da3
f928ade
0925965
db5ae3f
0c3e20c
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 |
|---|---|---|
| @@ -0,0 +1,77 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.execution.streaming | ||
|
|
||
| import java.io.{InputStream, OutputStream} | ||
| import java.nio.charset.StandardCharsets._ | ||
|
|
||
| import scala.io.{Source => IOSource} | ||
|
|
||
| import org.apache.spark.sql.SparkSession | ||
|
|
||
| /** | ||
| * Used to write log files that represent batch commit points in structured streaming. | ||
| * A commit log file will be written immediately after the successful completion of a | ||
| * batch, and before processing the next batch. Here is an execution summary: | ||
| * - trigger batch 1 | ||
| * - obtain batch 1 offsets and write to offset log | ||
| * - process batch 1 | ||
| * - write batch 1 to completion log | ||
| * - trigger batch 2 | ||
| * - obtain bactch 2 offsets and write to offset log | ||
| * - process batch 2 | ||
| * - write batch 2 to completion log | ||
| * .... | ||
| * | ||
| * The current format of the batch completion log is: | ||
| * line 1: version | ||
| * line 2: metadata (optional json string) | ||
| */ | ||
| class BatchCommitLog(sparkSession: SparkSession, path: String) | ||
| extends HDFSMetadataLog[String](sparkSession, path) { | ||
|
|
||
| override protected def deserialize(in: InputStream): String = { | ||
| // called inside a try-finally where the underlying stream is closed in the caller | ||
| val lines = IOSource.fromInputStream(in, UTF_8.name()).getLines() | ||
| if (!lines.hasNext) { | ||
| throw new IllegalStateException("Incomplete log file in the offset commit log") | ||
| } | ||
| parseVersion(lines.next().trim, BatchCommitLog.VERSION) | ||
| // read metadata | ||
| lines.next().trim match { | ||
| case BatchCommitLog.SERIALIZED_VOID => null | ||
| case metadata => metadata | ||
| } | ||
| } | ||
|
|
||
| override protected def serialize(metadata: String, out: OutputStream): Unit = { | ||
| // called inside a try-finally where the underlying stream is closed in the caller | ||
| out.write(s"v${BatchCommitLog.VERSION}".getBytes(UTF_8)) | ||
| out.write('\n') | ||
|
|
||
| // write metadata or void | ||
| out.write((if (metadata == null) BatchCommitLog.SERIALIZED_VOID else metadata) | ||
| .getBytes(UTF_8)) | ||
| } | ||
| } | ||
|
|
||
| object BatchCommitLog { | ||
| private val VERSION = 1 | ||
| private val SERIALIZED_VOID = "{}" | ||
| } | ||
|
|
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -165,6 +165,8 @@ class StreamExecution( | |
|
|
||
| private val triggerExecutor = trigger match { | ||
| case t: ProcessingTime => ProcessingTimeExecutor(t, triggerClock) | ||
| case OneTimeTrigger => OneTimeExecutor() | ||
| case _ => throw new IllegalStateException(s"Unknown type of trigger: $trigger") | ||
| } | ||
|
|
||
| /** Defines the internal state of execution */ | ||
|
|
@@ -209,6 +211,13 @@ class StreamExecution( | |
| */ | ||
| val offsetLog = new OffsetSeqLog(sparkSession, checkpointFile("offsets")) | ||
|
|
||
| /** | ||
| * A log that records the batch ids that have completed. This is used to check if a batch was | ||
| * fully processed, and its output was committed to the sink, hence no need to process it again. | ||
| * This is used (for instance) during restart, to help identify which batch to run next. | ||
| */ | ||
| val batchCommitLog = new BatchCommitLog(sparkSession, checkpointFile("commits")) | ||
|
|
||
| /** Whether all fields of the query have been initialized */ | ||
| private def isInitialized: Boolean = state.get != INITIALIZING | ||
|
|
||
|
|
@@ -291,10 +300,13 @@ class StreamExecution( | |
| runBatch(sparkSessionToRunBatches) | ||
| } | ||
| } | ||
|
|
||
| // Report trigger as finished and construct progress object. | ||
| finishTrigger(dataAvailable) | ||
| if (dataAvailable) { | ||
| // Update committed offsets. | ||
| committedOffsets ++= availableOffsets | ||
| batchCommitLog.add(currentBatchId, null) | ||
| logDebug(s"batch ${currentBatchId} committed") | ||
| // We'll increase currentBatchId after we complete processing current batch's data | ||
| currentBatchId += 1 | ||
| } else { | ||
|
|
@@ -306,9 +318,6 @@ class StreamExecution( | |
| } else { | ||
| false | ||
| } | ||
|
|
||
| // Update committed offsets. | ||
| committedOffsets ++= availableOffsets | ||
| updateStatusMessage("Waiting for next trigger") | ||
| continueToRun | ||
| }) | ||
|
|
@@ -392,13 +401,33 @@ class StreamExecution( | |
| * - currentBatchId | ||
| * - committedOffsets | ||
| * - availableOffsets | ||
| * The basic structure of this method is as follows: | ||
| * | ||
| * Identify (from the offset log) the offsets used to run the last batch | ||
| * IF last batch exists THEN | ||
| * Set the next batch to be executed as the last recovered batch | ||
| * Check the commit log to see which batch was committed last | ||
| * IF the last batch was committed THEN | ||
| * Call getBatch using the last batch start and end offsets | ||
| * // ^^^^ above line is needed since some sources assume last batch always re-executes | ||
| * Setup for a new batch i.e., start = last batch end, and identify new end | ||
| * DONE | ||
| * ELSE | ||
| * Identify a brand new batch | ||
| * DONE | ||
| */ | ||
| private def populateStartOffsets(sparkSessionToRunBatches: SparkSession): Unit = { | ||
| offsetLog.getLatest() match { | ||
| case Some((batchId, nextOffsets)) => | ||
| logInfo(s"Resuming streaming query, starting with batch $batchId") | ||
| currentBatchId = batchId | ||
| case Some((latestBatchId, nextOffsets)) => | ||
| /* First assume that we are re-executing the latest known batch | ||
| * in the offset log */ | ||
| currentBatchId = latestBatchId | ||
| availableOffsets = nextOffsets.toStreamProgress(sources) | ||
| /* Initialize committed offsets to a committed batch, which at this | ||
| * is the second latest batch id in the offset log. */ | ||
| offsetLog.get(latestBatchId - 1).foreach { secondLatestBatchId => | ||
| committedOffsets = secondLatestBatchId.toStreamProgress(sources) | ||
| } | ||
|
|
||
| // update offset metadata | ||
| nextOffsets.metadata.foreach { metadata => | ||
|
|
@@ -419,14 +448,37 @@ class StreamExecution( | |
| SQLConf.SHUFFLE_PARTITIONS.key, shufflePartitionsToUse.toString) | ||
| } | ||
|
|
||
| logDebug(s"Found possibly unprocessed offsets $availableOffsets " + | ||
| s"at batch timestamp ${offsetSeqMetadata.batchTimestampMs}") | ||
|
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 remove this debug log?
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. Yeah. this does not make sense here any more. But please add similar logging of recovered metadata later, where you have logged the start and available offsets. |
||
|
|
||
| offsetLog.get(batchId - 1).foreach { | ||
| case lastOffsets => | ||
| committedOffsets = lastOffsets.toStreamProgress(sources) | ||
| logDebug(s"Resuming with committed offsets: $committedOffsets") | ||
| /* identify the current batch id: if commit log indicates we successfully processed the | ||
| * latest batch id in the offset log, then we can safely move to the next batch | ||
| * i.e., committedBatchId + 1 */ | ||
| batchCommitLog.getLatest() match { | ||
| case Some((latestCommittedBatchId, _)) => | ||
| if (latestBatchId == latestCommittedBatchId) { | ||
| /* The last batch was successfully committed, so we can safely process a | ||
| * new next batch but first: | ||
| * Make a call to getBatch using the offsets from previous batch. | ||
| * because certain sources (e.g., KafkaSource) assume on restart the last | ||
| * batch will be executed before getOffset is called again. */ | ||
| availableOffsets.foreach { ao: (Source, Offset) => | ||
| val (source, end) = ao | ||
| if (committedOffsets.get(source).map(_ != end).getOrElse(true)) { | ||
| val start = committedOffsets.get(source) | ||
| source.getBatch(start, end) | ||
| } | ||
| } | ||
| currentBatchId = latestCommittedBatchId + 1 | ||
| committedOffsets ++= availableOffsets | ||
| // Construct a new batch be recomputing availableOffsets | ||
| constructNextBatch() | ||
| } else if (latestCommittedBatchId < latestBatchId - 1) { | ||
| logWarning(s"Batch completion log latest batch id is " + | ||
| s"${latestCommittedBatchId}, which is not trailing " + | ||
| s"batchid $latestBatchId by one") | ||
| } | ||
| case None => logInfo("no commit log present") | ||
| } | ||
| logDebug(s"Resuming at batch $currentBatchId with committed offsets " + | ||
| s"$committedOffsets and available offsets $availableOffsets") | ||
| case None => // We are starting this stream for the first time. | ||
| logInfo(s"Starting new streaming query.") | ||
| currentBatchId = 0 | ||
|
|
@@ -523,6 +575,7 @@ class StreamExecution( | |
| // Note that purge is exclusive, i.e. it purges everything before the target ID. | ||
| if (minBatchesToRetain < currentBatchId) { | ||
| offsetLog.purge(currentBatchId - minBatchesToRetain) | ||
| batchCommitLog.purge(currentBatchId - minBatchesToRetain) | ||
| } | ||
| } | ||
| } else { | ||
|
|
||
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.
really like this explanation.