-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-12817] Add BlockManager.getOrElseUpdate and remove CacheManager #11436
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
31e2ec3
d6ce63d
e5f505e
2613038
0c48c63
8f6cc09
e75e09b
2262dc3
d0f8685
50f66d1
f8cccea
dcd5f74
5a1fb12
51e1b2e
0aafba2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
This file was deleted.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -37,7 +37,7 @@ import org.apache.spark.partial.BoundedDouble | |
| import org.apache.spark.partial.CountEvaluator | ||
| import org.apache.spark.partial.GroupedCountEvaluator | ||
| import org.apache.spark.partial.PartialResult | ||
| import org.apache.spark.storage.StorageLevel | ||
| import org.apache.spark.storage.{RDDBlockId, StorageLevel} | ||
| import org.apache.spark.util.{BoundedPriorityQueue, Utils} | ||
| import org.apache.spark.util.collection.OpenHashMap | ||
| import org.apache.spark.util.random.{BernoulliCellSampler, BernoulliSampler, PoissonSampler, | ||
|
|
@@ -272,7 +272,7 @@ abstract class RDD[T: ClassTag]( | |
| */ | ||
| final def iterator(split: Partition, context: TaskContext): Iterator[T] = { | ||
| if (storageLevel != StorageLevel.NONE) { | ||
| SparkEnv.get.cacheManager.getOrCompute(this, split, context, storageLevel) | ||
| getOrCompute(split, context) | ||
| } else { | ||
| computeOrReadCheckpoint(split, context) | ||
| } | ||
|
|
@@ -314,6 +314,35 @@ abstract class RDD[T: ClassTag]( | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. | ||
| */ | ||
| private[spark] def getOrCompute(partition: Partition, context: TaskContext): Iterator[T] = { | ||
| val blockId = RDDBlockId(id, partition.index) | ||
| var readCachedBlock = true | ||
| // This method is called on executors, so we need call SparkEnv.get instead of sc.env. | ||
| SparkEnv.get.blockManager.getOrElseUpdate(blockId, storageLevel, () => { | ||
| readCachedBlock = false | ||
| computeOrReadCheckpoint(partition, context) | ||
| }) match { | ||
| case Left(blockResult) => | ||
| if (readCachedBlock) { | ||
| val existingMetrics = context.taskMetrics().registerInputMetrics(blockResult.readMethod) | ||
| existingMetrics.incBytesReadInternal(blockResult.bytes) | ||
| new InterruptibleIterator[T](context, blockResult.data.asInstanceOf[Iterator[T]]) { | ||
| override def next(): T = { | ||
| existingMetrics.incRecordsReadInternal(1) | ||
| delegate.next() | ||
|
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. crazy idea: what if we push all of this metrics code into
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. I considered this but had some reservations due to the fact that we might not want to consider every read of a BlockManager iterator to be a read of an input record (since we also read from iterators in the implementation of getSingle() and in a couple of other places, and those methods are used for things like TorrentBroadcast blocks which I don't think we want to treat as input records in for the purposes of these metrics). On the other hand, this |
||
| } | ||
| } | ||
| } else { | ||
| new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]]) | ||
| } | ||
| case Right(iter) => | ||
| new InterruptibleIterator(context, iter.asInstanceOf[Iterator[T]]) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Execute a block of code in a scope such that all new RDDs created in this body will | ||
| * be part of the same scope. For more detail, see {{org.apache.spark.rdd.RDDOperationScope}}. | ||
|
|
||
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.
Can you add that this is only called on the executors? (otherwise people might wonder why we don't just use
sc.envinstead ofSparkEnv.get)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.
Done.