-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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-18406][CORE] Race between end-of-task and completion iterator read lock release #18076
Changes from 1 commit
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 |
---|---|---|
|
@@ -501,14 +501,18 @@ private[spark] class BlockManager( | |
case Some(info) => | ||
val level = info.level | ||
logDebug(s"Level for block $blockId is $level") | ||
val taskAttemptId = Option(TaskContext.get()).map(_.taskAttemptId()) | ||
.getOrElse(BlockInfo.NON_TASK_WRITER) | ||
if (level.useMemory && memoryStore.contains(blockId)) { | ||
val iter: Iterator[Any] = if (level.deserialized) { | ||
memoryStore.getValues(blockId).get | ||
} else { | ||
serializerManager.dataDeserializeStream( | ||
blockId, memoryStore.getBytes(blockId).get.toInputStream())(info.classTag) | ||
} | ||
val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId)) | ||
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. I'd add a one-line comment before this line which references SPARK-18406, something like
or similar. |
||
val ci = CompletionIterator[Any, Iterator[Any]](iter, { | ||
releaseLock(blockId, Some(taskAttemptId)) | ||
}) | ||
Some(new BlockResult(ci, DataReadMethod.Memory, info.size)) | ||
} else if (level.useDisk && diskStore.contains(blockId)) { | ||
val diskData = diskStore.getBytes(blockId) | ||
|
@@ -525,8 +529,9 @@ private[spark] class BlockManager( | |
serializerManager.dataDeserializeStream(blockId, stream)(info.classTag) | ||
} | ||
} | ||
val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, | ||
releaseLockAndDispose(blockId, diskData)) | ||
val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, { | ||
releaseLockAndDispose(blockId, diskData, Some(taskAttemptId)) | ||
}) | ||
Some(new BlockResult(ci, DataReadMethod.Disk, info.size)) | ||
} else { | ||
handleLocalReadFailure(blockId) | ||
|
@@ -713,8 +718,15 @@ private[spark] class BlockManager( | |
/** | ||
* Release a lock on the given block. | ||
*/ | ||
def releaseLock(blockId: BlockId): Unit = { | ||
blockInfoManager.unlock(blockId) | ||
def releaseLock(blockId: BlockId): Unit = releaseLock(blockId, taskAttemptId = None) | ||
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 do we need to overload here? Why not just have a single 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. In fact 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. I think there's only one implementation of |
||
|
||
/** | ||
* Release a lock on the given block with explicit TID. | ||
* This method should be used in case we can't get the correct TID from TaskContext, for example, | ||
* the input iterator of a cached RDD iterates to the end in a child thread. | ||
*/ | ||
def releaseLock(blockId: BlockId, taskAttemptId: Option[Long]): Unit = { | ||
blockInfoManager.unlock(blockId, taskAttemptId) | ||
} | ||
|
||
/** | ||
|
@@ -1467,8 +1479,11 @@ private[spark] class BlockManager( | |
} | ||
} | ||
|
||
def releaseLockAndDispose(blockId: BlockId, data: BlockData): Unit = { | ||
blockInfoManager.unlock(blockId) | ||
def releaseLockAndDispose( | ||
blockId: BlockId, | ||
data: BlockData, | ||
taskAttemptId: Option[Long] = None): Unit = { | ||
releaseLock(blockId, taskAttemptId) | ||
data.dispose() | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -30,7 +30,7 @@ import org.apache.hadoop.mapred.{FileSplit, TextInputFormat} | |
import org.apache.spark._ | ||
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} | ||
import org.apache.spark.rdd.RDDSuiteUtils._ | ||
import org.apache.spark.util.Utils | ||
import org.apache.spark.util.{ThreadUtils, Utils} | ||
|
||
class RDDSuite extends SparkFunSuite with SharedSparkContext { | ||
var tempDir: File = _ | ||
|
@@ -1082,6 +1082,22 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { | |
assert(totalPartitionCount == 10) | ||
} | ||
|
||
test("SPARK-18406: race between end-of-task and completion iterator read lock release") { | ||
val rdd = sc.parallelize(1 to 1000, 10) | ||
rdd.cache() | ||
|
||
rdd.mapPartitions { iter => | ||
ThreadUtils.runInNewThread("TestThread") { | ||
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. Nice use of this helper method. I wasn't aware of this, but it's pretty nice. I'll use it in my own tests going forward. |
||
// Iterate to the end of the input iterator, to cause the CompletionIterator completion to | ||
// fire outside of the task's main thread. | ||
while (iter.hasNext) { | ||
iter.next() | ||
} | ||
iter | ||
} | ||
}.collect() | ||
} | ||
|
||
// NOTE | ||
// Below tests calling sc.stop() have to be the last tests in this suite. If there are tests | ||
// running after them and if they access sc those tests will fail as sc is already closed, because | ||
|
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 think we can leave out the
.getOrElse
here and just pass in theOption
itself intoreleaseLock
. This helps to avoid exposure ofBlockInfo.NON_TASK_WRITER
here. Not a huge deal but just a minor nit.