-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-20638][Core]Optimize the CartesianRDD to reduce repeatedly data fetching #17936
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 8 commits
1802ff0
08e25c9
0f812d9
08c1849
89a22ef
397dd90
697ba33
c8222f4
f29a9dc
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 |
|---|---|---|
|
|
@@ -22,7 +22,8 @@ import java.io.{IOException, ObjectOutputStream} | |
| import scala.reflect.ClassTag | ||
|
|
||
| import org.apache.spark._ | ||
| import org.apache.spark.util.Utils | ||
| import org.apache.spark.storage.{RDDBlockId, StorageLevel} | ||
| import org.apache.spark.util.{CompletionIterator, Utils} | ||
|
|
||
| private[spark] | ||
| class CartesianPartition( | ||
|
|
@@ -71,9 +72,92 @@ class CartesianRDD[T: ClassTag, U: ClassTag]( | |
| } | ||
|
|
||
| override def compute(split: Partition, context: TaskContext): Iterator[(T, U)] = { | ||
| val blockManager = SparkEnv.get.blockManager | ||
| val currSplit = split.asInstanceOf[CartesianPartition] | ||
| for (x <- rdd1.iterator(currSplit.s1, context); | ||
| y <- rdd2.iterator(currSplit.s2, context)) yield (x, y) | ||
| val blockId2 = RDDBlockId(rdd2.id, currSplit.s2.index) | ||
| var cachedInLocal = false | ||
| var holdReadLock = false | ||
|
|
||
| // Try to get data from the local, otherwise it will be cached to the local. | ||
| def getOrElseCache( | ||
| rdd: RDD[U], | ||
| partition: Partition, | ||
| context: TaskContext, | ||
| level: StorageLevel): Iterator[U] = { | ||
| getLocalValues() match { | ||
| case Some(result) => | ||
| return result | ||
| case None => if (holdReadLock) { | ||
| throw new SparkException(s"get() failed for block $blockId2 even though we held a lock") | ||
| } | ||
| } | ||
|
|
||
| val iterator = rdd.iterator(partition, context) | ||
| if (rdd.getStorageLevel != StorageLevel.NONE || rdd.isCheckpointedAndMaterialized) { | ||
| // If the block is cached in local, wo shouldn't cache it again. | ||
|
||
| return iterator | ||
| } | ||
|
|
||
| // Keep read lock, because next we need read it. And don't tell master. | ||
| val putSuccess = blockManager.putIterator[U](blockId2, iterator, level, false, true) | ||
| if (putSuccess) { | ||
| cachedInLocal = true | ||
| // After we cached the block, we also hold the block read lock until this task finished. | ||
| holdReadLock = true | ||
| logInfo(s"Cache the block $blockId2 to local successful.") | ||
| val readLocalBlock = blockManager.getLocalValues(blockId2).getOrElse { | ||
| blockManager.releaseLock(blockId2) | ||
| throw new SparkException(s"get() failed for block $blockId2 even though we held a lock") | ||
| } | ||
|
|
||
| new InterruptibleIterator[U](context, readLocalBlock.data.asInstanceOf[Iterator[U]]) | ||
| } else { | ||
| blockManager.releaseLock(blockId2) | ||
| // There shouldn't a error caused by put in memory, because we use MEMORY_AND_DISK to | ||
| // cache it. | ||
| throw new SparkException(s"Cache block $blockId2 in local failed even though it's $level") | ||
| } | ||
| } | ||
|
|
||
| // Get block from local, and update the metrics. | ||
| def getLocalValues(): Option[Iterator[U]] = { | ||
| blockManager.getLocalValues(blockId2) match { | ||
| case Some(result) => | ||
| val existingMetrics = context.taskMetrics().inputMetrics | ||
| existingMetrics.incBytesRead(result.bytes) | ||
| val localIter = | ||
| new InterruptibleIterator[U](context, result.data.asInstanceOf[Iterator[U]]) { | ||
| override def next(): U = { | ||
| existingMetrics.incRecordsRead(1) | ||
| delegate.next() | ||
| } | ||
| } | ||
| Some(localIter) | ||
| case None => | ||
| None | ||
| } | ||
| } | ||
|
|
||
| def removeCachedBlock(): Unit = { | ||
| val blockManager = SparkEnv.get.blockManager | ||
| if (holdReadLock) { | ||
| // If hold the read lock, we need release it. | ||
| blockManager.releaseLock(blockId2) | ||
| } | ||
| // Whether the block it persisted by the user. | ||
| val persistedInLocal = | ||
| blockManager.master.getLocations(blockId2).contains(blockManager.blockManagerId) | ||
| if (!persistedInLocal && (cachedInLocal || blockManager.isRemovable(blockId2))) { | ||
| blockManager.removeOrMarkAsRemovable(blockId2, false) | ||
| } | ||
| } | ||
|
|
||
| val resultIter = | ||
| for (x <- rdd1.iterator(currSplit.s1, context); | ||
| y <- getOrElseCache(rdd2, currSplit.s2, context, StorageLevel.MEMORY_AND_DISK)) | ||
| yield (x, y) | ||
|
|
||
| CompletionIterator[(T, U), Iterator[(T, U)]](resultIter, removeCachedBlock()) | ||
| } | ||
|
|
||
| override def getDependencies: Seq[Dependency[_]] = List( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -198,8 +198,12 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext | |
| // write files to disk so we can read them later. | ||
| sc.parallelize(cartVector).saveAsTextFile(cartFilePath) | ||
| val aRdd = sc.textFile(cartFilePath, numPartitions) | ||
| aRdd.cache() | ||
| aRdd.count() | ||
|
||
|
|
||
| val tmpRdd = sc.textFile(tmpFilePath, numPartitions) | ||
| tmpRdd.cache() | ||
| tmpRdd.count() | ||
|
||
|
|
||
| val firstSize = runAndReturnBytesRead { | ||
| aRdd.count() | ||
|
|
||
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.
Btw, can we move those functions out of
compute? Too many nested functions here and makingcomputetoo big.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.
Ok, I will change it too.