diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
index 43ee39c993a3c0b254fe35bbbf9bcd8fb167e10c..377b1bdbe002fc37b3a68d475e2b391f3537175d 100644
--- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
@@ -43,7 +43,7 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri
   checkpointData.get.cpFile = Some(checkpointPath)
 
   override def getPreferredLocations(split: Partition): Seq[String] = {
-    val status = fs.getFileStatus(new Path(checkpointPath))
+    val status = fs.getFileStatus(new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index)))
     val locations = fs.getFileBlockLocations(status, 0, status.getLen)
     locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost")
   }