-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-16963] [STREAMING] [SQL] Changes to Source trait and related implementation classes #14553
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
Closed
Closed
Changes from 1 commit
Commits
Show all changes
31 commits
Select commit
Hold shift + click to select a range
6c9acde
Initial version of changes to Source trait
frreiss dae72ff
Changes to files that depend on the Source trait
frreiss f78b4d5
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss cf426fa
Added method to garbage-collect the metadata log.
frreiss c028432
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss f92a9a7
Fixing problems with building from Maven.
frreiss 4cd181d
Various bug fixes.
frreiss fcc90bd
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss 35cdae9
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss 9096c56
Merge branch 'master' of https://github.com/apache/spark into fred-16…
frreiss ecaf732
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss 5638281
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss 43ffbf3
Removed a few blank lines.
frreiss f5c15f8
Additional whitespace cleanup.
frreiss a79c557
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss 7c6a30d
Narrowing the size of the diff by moving some changes out to future w…
frreiss 5e340c2
Fixed a regression introduced in an earlier merge.
frreiss 128f7fe
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss 6334a4b
Fixed compilation problem from merging someone else's PR.
frreiss 09e4b8e
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss aaf0307
Removed a safety check that was invalidated by SPARK-17643 and fixed …
frreiss 947b510
Updating regression tests after merge.
frreiss ed887ca
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss ec67429
Changes to address review comments.
frreiss e7ef7ab
Fix compilation problems.
frreiss 7d98c6b
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss c726549
Changes to address review comments.
frreiss 47eee52
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss 46f6411
Commit before merge.
frreiss d9eaf5a
Merge branch 'master' of https://github.com/apache/spark into fred-16963
frreiss 0a56e4a
Addressing review comments.
frreiss File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Changes to files that depend on the Source trait
- Loading branch information
commit dae72ff923edbbeb0f0ac2fd251a25d8372ad5fa
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,7 +21,9 @@ import java.util.concurrent.{CountDownLatch, TimeUnit} | |
| import java.util.concurrent.atomic.AtomicLong | ||
| import java.util.concurrent.locks.ReentrantLock | ||
|
|
||
| import scala.collection.mutable | ||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.collection.mutable.Map | ||
| import scala.util.control.NonFatal | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
|
|
@@ -72,13 +74,17 @@ class StreamExecution( | |
| /** | ||
| * Tracks how much data we have processed and committed to the sink or state store from each | ||
| * input source. | ||
| * Only the scheduler thread should modify this field, and only in atomic steps. Other threads | ||
| * must create a local copy before iterating over this data structure. | ||
| */ | ||
| @volatile | ||
| private[sql] var committedOffsets = new StreamProgress | ||
|
|
||
| /** | ||
| * Tracks the offsets that are available to be processed, but have not yet be committed to the | ||
| * sink. | ||
| * Only the scheduler thread should modify this field, and only in atomic steps. Other threads | ||
| * must create a local copy before iterating over this data structure. | ||
| */ | ||
| @volatile | ||
| private var availableOffsets = new StreamProgress | ||
|
|
@@ -248,6 +254,21 @@ class StreamExecution( | |
| logDebug(s"Resuming with committed offsets: $committedOffsets") | ||
| } | ||
|
|
||
| // Compare the offsets we just read from the checkpoint against the | ||
| // sources' own checkpoint data. | ||
| val offsetChanges = mutable.Map[Source, Offset]() | ||
| committedOffsets.map { | ||
| case (src, checkptOffset) => | ||
| val srcOffset = src.getMinOffset | ||
| if (srcOffset.isDefined && srcOffset.get > checkptOffset) { | ||
| logWarning(s"Source $src lost offsets between $checkptOffset " + | ||
| s"and $srcOffset when resuming. Skipping ahead to $srcOffset.") | ||
| offsetChanges += (src -> srcOffset.get) | ||
| } | ||
| } | ||
| committedOffsets ++= offsetChanges | ||
|
|
||
|
|
||
|
||
| case None => // We are starting this stream for the first time. | ||
| logInfo(s"Starting new streaming query.") | ||
| currentBatchId = 0 | ||
|
|
@@ -277,7 +298,7 @@ class StreamExecution( | |
| val hasNewData = { | ||
| awaitBatchLock.lock() | ||
| try { | ||
| val newData = uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) | ||
| val newData = uniqueSources.flatMap(s => s.getMaxOffset.map(o => s -> o)) | ||
| availableOffsets ++= newData | ||
|
|
||
| if (dataAvailable) { | ||
|
|
@@ -294,6 +315,12 @@ class StreamExecution( | |
| assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), | ||
| s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") | ||
| logInfo(s"Committed offsets for batch $currentBatchId.") | ||
|
|
||
| // Now that we've updated the scheduler's persistent checkpoint, it is safe for the | ||
| // sources to discard batches from the *previous* batch. | ||
| committedOffsets.foreach { | ||
| case (src, off) => src.commit(off) | ||
| } | ||
| } else { | ||
| awaitBatchLock.lock() | ||
| try { | ||
|
|
@@ -374,6 +401,8 @@ class StreamExecution( | |
| logInfo(s"Completed up to $availableOffsets in ${batchTime}ms") | ||
| // Update committed offsets. | ||
| committedOffsets ++= availableOffsets | ||
|
|
||
|
|
||
| postEvent(new QueryProgress(this.toInfo)) | ||
| } | ||
|
|
||
|
|
@@ -399,7 +428,7 @@ class StreamExecution( | |
|
|
||
| /** | ||
| * Blocks the current thread until processing for data from the given `source` has reached at | ||
| * least the given `Offset`. This method is indented for use primarily when writing tests. | ||
| * least the given `Offset`. This method is intended for use primarily when writing tests. | ||
| */ | ||
| def awaitOffset(source: Source, newOffset: Offset): Unit = { | ||
| def notDone = { | ||
|
|
||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
I agree with the first sentance. I'm not sure I understand the second.
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.
What I meant to say is that two accesses to this field from a thread other than the scheduler thread may return different different values. The functions that access
committedOffsetsfrom other threads either should access the field only once (toDebugString); or they should make a copy (as in the code that consumesStreamingQueryStatusobjects). Updated the comment in my local copy to something that is hopefully more clear: "Other threads should make a shallow copy if they are going to access this field more than once, since the field's value may change at any time."