You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/10/14 22:23:25 UTC

[GitHub] [spark] squito commented on a change in pull request #25856: [SPARK-29182][Core] Cache preferred locations of checkpointed RDD

squito commented on a change in pull request #25856: [SPARK-29182][Core] Cache preferred locations of checkpointed RDD
URL: https://github.com/apache/spark/pull/25856#discussion_r334679728
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala
 ##########
 @@ -82,16 +84,38 @@ private[spark] class ReliableCheckpointRDD[T: ClassTag](
     Array.tabulate(inputFiles.length)(i => new CheckpointRDDPartition(i))
   }
 
-  /**
-   * Return the locations of the checkpoint file associated with the given partition.
-   */
-  protected override def getPreferredLocations(split: Partition): Seq[String] = {
+  // Cache of preferred locations of checkpointed files.
+  @transient private[spark] lazy val cachedPreferredLocations = CacheBuilder.newBuilder()
+    .expireAfterWrite(
+      SparkEnv.get.conf.get(CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME).get,
+      TimeUnit.MINUTES)
+    .build(
+      new CacheLoader[Partition, Seq[String]]() {
+        override def load(split: Partition): Seq[String] = {
+          getPartitionBlockLocations(split)
+        }
+      })
+
+  // Returns the block locations of given partition on file system.
+  private def getPartitionBlockLocations(split: Partition): Seq[String] = {
     val status = fs.getFileStatus(
       new Path(checkpointPath, ReliableCheckpointRDD.checkpointFileName(split.index)))
     val locations = fs.getFileBlockLocations(status, 0, status.getLen)
     locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost")
   }
 
+  /**
+   * Return the locations of the checkpoint file associated with the given partition.
+   */
+  protected override def getPreferredLocations(split: Partition): Seq[String] = {
+    val cachedExpireTime = SparkEnv.get.conf.get(CACHE_CHECKPOINT_PREFERRED_LOCS_EXPIRE_TIME)
 
 Review comment:
   minor -- reading confs is a bit expensive, you should do this once for the rdd, rather than once per call

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org