-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-18928][branch-2.0]Check TaskContext.isInterrupted() in FileScanRDD, JDBCRDD & UnsafeSorter #16357
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
[SPARK-18928][branch-2.0]Check TaskContext.isInterrupted() in FileScanRDD, JDBCRDD & UnsafeSorter #16357
Changes from all commits
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 |
|---|---|---|
|
|
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources | |
|
|
||
| import scala.collection.mutable | ||
|
|
||
| import org.apache.spark.{Partition => RDDPartition, TaskContext} | ||
| import org.apache.spark.{Partition => RDDPartition, TaskContext, TaskKilledException} | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.rdd.{InputFileNameHolder, RDD} | ||
| import org.apache.spark.sql.SparkSession | ||
|
|
@@ -88,7 +88,15 @@ class FileScanRDD( | |
| private[this] var currentFile: PartitionedFile = null | ||
| private[this] var currentIterator: Iterator[Object] = null | ||
|
|
||
| def hasNext = (currentIterator != null && currentIterator.hasNext) || nextIterator() | ||
| def hasNext: Boolean = { | ||
| // Kill the task in case it has been marked as killed. This logic is from | ||
| // InterruptibleIterator, but we inline it here instead of wrapping the iterator in order | ||
| // to avoid performance overhead. | ||
| if (context.isInterrupted()) { | ||
| throw new TaskKilledException | ||
| } | ||
|
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 not move this to next() instead of hasNext (latter is not mandatory to be called - as seen here : #16252)
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. This particular iterator already won't work unless
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. Yes, looks like this iterator is already broken; and we are adding to that now. |
||
| (currentIterator != null && currentIterator.hasNext) || nextIterator() | ||
| } | ||
| def next() = { | ||
| val nextElement = currentIterator.next() | ||
| // TODO: we should have a better separation of row based and batch based scan, so that we | ||
|
|
||
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.
Wont this not be in the internal tight loop for reading data ?
If yes, dereferencing a volatile for each tuple processed is worrying.
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 already have this in tight loops in the form of
InterruptibleIteratorwrapping all over the place.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.
In an admittedly non-scientific benchmark, I tried running
a few times with and without the
tc.isInterrupted()check and there wasn't a measurable time difference in my environment. While I imagine that the volatile read could incur some higher costs in certain circumstances I think that the overhead of all of the virtual function calls and iterator interfaces, etc. will mask any gains by optimizing this read.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.
As you mentioned, this is unscientific microbenchmark :-)
Also, the isInterrupted was probably optimized away anyway ?
Since we dont need gaurantees on how soon interruption is to be honoured, batching its application (if possible) would be better in sorting ?
Yes, we do have InterruptibleIterator - unfortunately, we dont have a way to optimize that (afaik).