diff --git a/core/src/main/scala/spark/BlockRDD.scala b/core/src/main/scala/spark/BlockRDD.scala
index afc732234f10578a5a73e44d70748a2b6274814f..cb73976aedd35daddc1d4e64e8ecbd6a7a38873a 100644
--- a/core/src/main/scala/spark/BlockRDD.scala
+++ b/core/src/main/scala/spark/BlockRDD.scala
@@ -1,7 +1,13 @@
-package spark
+package spark.rdd
 
 import scala.collection.mutable.HashMap
 
+import spark.Dependency
+import spark.RDD
+import spark.SparkContext
+import spark.SparkEnv
+import spark.Split
+
 private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split {
   val index = idx
 }
diff --git a/core/src/main/scala/spark/CartesianRDD.scala b/core/src/main/scala/spark/CartesianRDD.scala
index 52aab5f32ff4904c7d7edf3a8ad1995b44876797..7c354b6b2eccd13a5107eb26b772d5694bef7037 100644
--- a/core/src/main/scala/spark/CartesianRDD.scala
+++ b/core/src/main/scala/spark/CartesianRDD.scala
@@ -1,4 +1,9 @@
-package spark
+package spark.rdd
+
+import spark.NarrowDependency
+import spark.RDD
+import spark.SparkContext
+import spark.Split
 
 private[spark]
 class CartesianSplit(idx: Int, val s1: Split, val s2: Split) extends Split with Serializable {
diff --git a/core/src/main/scala/spark/CoGroupedRDD.scala b/core/src/main/scala/spark/CoGroupedRDD.scala
index daba719b14a3871309ebf5840174f21a84e8a7a8..8fa0749184f8cbfe29833e869e6c75f3886771b4 100644
--- a/core/src/main/scala/spark/CoGroupedRDD.scala
+++ b/core/src/main/scala/spark/CoGroupedRDD.scala
@@ -1,11 +1,22 @@
-package spark
+package spark.rdd
 
 import java.net.URL
 import java.io.EOFException
 import java.io.ObjectInputStream
+
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 
+import spark.Aggregator
+import spark.Dependency
+import spark.Logging
+import spark.OneToOneDependency
+import spark.Partitioner
+import spark.RDD
+import spark.ShuffleDependency
+import spark.SparkEnv
+import spark.Split
+
 private[spark] sealed trait CoGroupSplitDep extends Serializable
 private[spark] case class NarrowCoGroupSplitDep(rdd: RDD[_], split: Split) extends CoGroupSplitDep
 private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
diff --git a/core/src/main/scala/spark/CoalescedRDD.scala b/core/src/main/scala/spark/CoalescedRDD.scala
index f1ae346a445a89a5e3d26b3d062fcf4e795652ac..0967f4f5df8812f609de8f1ca440265897463725 100644
--- a/core/src/main/scala/spark/CoalescedRDD.scala
+++ b/core/src/main/scala/spark/CoalescedRDD.scala
@@ -1,4 +1,8 @@
-package spark
+package spark.rdd
+
+import spark.NarrowDependency
+import spark.RDD
+import spark.Split
 
 private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) extends Split
 
diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala
index 1fbf66b7ded3c2e16ed708159be075e12ea0e8e3..d232ddeb7c6c3b797d273b848f152b8cb3288972 100644
--- a/core/src/main/scala/spark/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala
@@ -1,10 +1,13 @@
-package spark
+package spark.rdd
 
 import spark.partial.BoundedDouble
 import spark.partial.MeanEvaluator
 import spark.partial.PartialResult
 import spark.partial.SumEvaluator
 
+import spark.Logging
+import spark.RDD
+import spark.TaskContext
 import spark.util.StatCounter
 
 /**
diff --git a/core/src/main/scala/spark/HadoopRDD.scala b/core/src/main/scala/spark/HadoopRDD.scala
index 6d448116a9583fd96b6cb700522409f8a7ed07f9..bf29a1f0750129222779b2485a516cf7db8dd79e 100644
--- a/core/src/main/scala/spark/HadoopRDD.scala
+++ b/core/src/main/scala/spark/HadoopRDD.scala
@@ -1,4 +1,4 @@
-package spark
+package spark.rdd
 
 import java.io.EOFException
 import java.util.NoSuchElementException
@@ -15,6 +15,12 @@ import org.apache.hadoop.mapred.RecordReader
 import org.apache.hadoop.mapred.Reporter
 import org.apache.hadoop.util.ReflectionUtils
 
+import spark.Dependency
+import spark.RDD
+import spark.SerializableWritable
+import spark.SparkContext
+import spark.Split
+
 /** 
  * A Spark split class that wraps around a Hadoop InputSplit.
  */
diff --git a/core/src/main/scala/spark/NewHadoopRDD.scala b/core/src/main/scala/spark/NewHadoopRDD.scala
index 9072698357fa7714597017228c127f2f82a1579f..dcbceab24653dd84a6a2b15e2f69bd7a5a1bcebf 100644
--- a/core/src/main/scala/spark/NewHadoopRDD.scala
+++ b/core/src/main/scala/spark/NewHadoopRDD.scala
@@ -1,4 +1,4 @@
-package spark
+package spark.rdd
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.Writable
@@ -13,6 +13,12 @@ import org.apache.hadoop.mapreduce.TaskAttemptID
 import java.util.Date
 import java.text.SimpleDateFormat
 
+import spark.Dependency
+import spark.RDD
+import spark.SerializableWritable
+import spark.SparkContext
+import spark.Split
+
 private[spark] 
 class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable)
   extends Split {
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index 80d62caf25ad6450552b9614a3367168a4188c2f..2a94ea263ae9999be1f4c9d5497d4d239aa474dc 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/spark/PairRDDFunctions.scala
@@ -1,4 +1,4 @@
-package spark
+package spark.rdd
 
 import java.io.EOFException
 import java.io.ObjectInputStream
@@ -34,9 +34,20 @@ import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob}
 import org.apache.hadoop.mapreduce.TaskAttemptID
 import org.apache.hadoop.mapreduce.TaskAttemptContext
 
-import spark.SparkContext._
 import spark.partial.BoundedDouble
 import spark.partial.PartialResult
+import spark.Aggregator
+import spark.HashPartitioner
+import spark.Logging
+import spark.OneToOneDependency
+import spark.Partitioner
+import spark.RangePartitioner
+import spark.RDD
+import spark.SerializableWritable
+import spark.SparkContext._
+import spark.SparkException
+import spark.Split
+import spark.TaskContext
 
 /**
  * Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
diff --git a/core/src/main/scala/spark/PipedRDD.scala b/core/src/main/scala/spark/PipedRDD.scala
index 3103d7889b00326c493b139a3861d8d5be194c68..98ea0c92d6b527cedd29e3b5ac58a8071042ea89 100644
--- a/core/src/main/scala/spark/PipedRDD.scala
+++ b/core/src/main/scala/spark/PipedRDD.scala
@@ -1,4 +1,4 @@
-package spark
+package spark.rdd
 
 import java.io.PrintWriter
 import java.util.StringTokenizer
@@ -8,6 +8,12 @@ import scala.collection.JavaConversions._
 import scala.collection.mutable.ArrayBuffer
 import scala.io.Source
 
+import spark.OneToOneDependency
+import spark.RDD
+import spark.SparkEnv
+import spark.Split
+
+
 /**
  * An RDD that pipes the contents of each parent partition through an external command
  * (printing them one per line) and returns the output as a collection of strings.
diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala
index 3244753bfe49b3c38c03e96e3ef8525e5746976e..f0e4fb643f7d3c2945c2c2ac83d7dc8d33d55c29 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/spark/RDD.scala
@@ -31,6 +31,11 @@ import spark.partial.BoundedDouble
 import spark.partial.CountEvaluator
 import spark.partial.GroupedCountEvaluator
 import spark.partial.PartialResult
+import spark.rdd.BlockRDD
+import spark.rdd.CartesianRDD
+import spark.rdd.PipedRDD
+import spark.rdd.SampledRDD
+import spark.rdd.UnionRDD
 import spark.storage.StorageLevel
 
 import SparkContext._
diff --git a/core/src/main/scala/spark/SampledRDD.scala b/core/src/main/scala/spark/SampledRDD.scala
index ac10aed4777cdaab4ed0572007d1655413c66995..87a5268f2780dc0369aaa51a3b0fe9aaa9731e44 100644
--- a/core/src/main/scala/spark/SampledRDD.scala
+++ b/core/src/main/scala/spark/SampledRDD.scala
@@ -1,9 +1,13 @@
-package spark
+package spark.rdd
 
 import java.util.Random
 import cern.jet.random.Poisson
 import cern.jet.random.engine.DRand
 
+import spark.RDD
+import spark.OneToOneDependency
+import spark.Split
+
 private[spark]
 class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Serializable {
   override val index: Int = prev.index
diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
index ea7171d3a126c99170b3fba6446a247c7970254f..24c731fa92bd98675c8d2c2fbb41a612d824ad86 100644
--- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
@@ -1,4 +1,4 @@
-package spark
+package spark.rdd
 
 import java.io.EOFException
 import java.net.URL
@@ -23,7 +23,9 @@ import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.io.BytesWritable
 import org.apache.hadoop.io.Text
 
-import SparkContext._
+import spark.Logging
+import spark.RDD
+import spark.SparkContext._
 
 /**
  * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile,
diff --git a/core/src/main/scala/spark/ShuffledRDD.scala b/core/src/main/scala/spark/ShuffledRDD.scala
index 1a9f4cfec39264339e554372e91f6278bb3a93fe..769ccf8caa32fb056081336bbad990b4b195ac49 100644
--- a/core/src/main/scala/spark/ShuffledRDD.scala
+++ b/core/src/main/scala/spark/ShuffledRDD.scala
@@ -1,8 +1,15 @@
-package spark
+package spark.rdd
 
 import scala.collection.mutable.ArrayBuffer
 import java.util.{HashMap => JHashMap}
 
+import spark.Aggregator
+import spark.Partitioner
+import spark.RangePartitioner
+import spark.RDD
+import spark.ShuffleDependency
+import spark.SparkEnv
+import spark.Split
 
 private[spark] class ShuffledRDDSplit(val idx: Int) extends Split {
   override val index = idx
diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index 37ba308546a967214e85a2728486e184f8115000..84fc541f82ef543f41d85cb8de91433f16bdd9a7 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -4,12 +4,11 @@ import java.io._
 import java.util.concurrent.atomic.AtomicInteger
 import java.net.{URI, URLClassLoader}
 
-import akka.actor.Actor
-import akka.actor.Actor._
-
 import scala.collection.mutable.{ArrayBuffer, HashMap}
 import scala.collection.generic.Growable
 
+import akka.actor.Actor
+import akka.actor.Actor._
 import org.apache.hadoop.fs.{FileUtil, Path}
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapred.InputFormat
@@ -27,20 +26,22 @@ import org.apache.hadoop.io.Text
 import org.apache.hadoop.mapred.FileInputFormat
 import org.apache.hadoop.mapred.JobConf
 import org.apache.hadoop.mapred.TextInputFormat
-
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
 import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
 import org.apache.hadoop.mapreduce.{Job => NewHadoopJob}
-
 import org.apache.mesos.{Scheduler, MesosNativeLibrary}
 
 import spark.broadcast._
-
 import spark.deploy.LocalSparkCluster
-
 import spark.partial.ApproximateEvaluator
 import spark.partial.PartialResult
-
+import spark.rdd.DoubleRDDFunctions
+import spark.rdd.HadoopRDD
+import spark.rdd.NewHadoopRDD
+import spark.rdd.OrderedRDDFunctions
+import spark.rdd.PairRDDFunctions
+import spark.rdd.SequenceFileRDDFunctions
+import spark.rdd.UnionRDD
 import spark.scheduler.ShuffleMapTask
 import spark.scheduler.DAGScheduler
 import spark.scheduler.TaskScheduler
diff --git a/core/src/main/scala/spark/UnionRDD.scala b/core/src/main/scala/spark/UnionRDD.scala
index 3e795ea2a242ed23cf5253b04a87b40b2ed4ac14..4ba2848491acf2cd7375cfc086f1a6145966b8b3 100644
--- a/core/src/main/scala/spark/UnionRDD.scala
+++ b/core/src/main/scala/spark/UnionRDD.scala
@@ -1,7 +1,13 @@
-package spark
+package spark.rdd
 
 import scala.collection.mutable.ArrayBuffer
 
+import spark.Dependency
+import spark.RangeDependency
+import spark.RDD
+import spark.SparkContext
+import spark.Split
+
 private[spark] class UnionSplit[T: ClassManifest](
     idx: Int, 
     rdd: RDD[T],
diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
index 84ec386ce4722a4e06af2dfeb6db3c4df73c0758..3c4399493c63b80800a41f2f47508111dda38496 100644
--- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
@@ -1,13 +1,5 @@
 package spark.api.java
 
-import spark.SparkContext.rddToPairRDDFunctions
-import spark.api.java.function.{Function2 => JFunction2}
-import spark.api.java.function.{Function => JFunction}
-import spark.partial.BoundedDouble
-import spark.partial.PartialResult
-import spark.storage.StorageLevel
-import spark._
-
 import java.util.{List => JList}
 import java.util.Comparator
 
@@ -19,6 +11,17 @@ import org.apache.hadoop.mapred.OutputFormat
 import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
 import org.apache.hadoop.conf.Configuration
 
+import spark.api.java.function.{Function2 => JFunction2}
+import spark.api.java.function.{Function => JFunction}
+import spark.partial.BoundedDouble
+import spark.partial.PartialResult
+import spark.rdd.OrderedRDDFunctions
+import spark.storage.StorageLevel
+import spark.HashPartitioner
+import spark.Partitioner
+import spark.RDD
+import spark.SparkContext.rddToPairRDDFunctions
+
 class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManifest[K],
   implicit val vManifest: ClassManifest[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] {
 
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala
index ade457c0f9b54a545d0d3142ed1f3a8a10482054..95e402627c6659ad4db6bbca2edaf9b40eb81230 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/spark/RDDSuite.scala
@@ -3,6 +3,8 @@ package spark
 import scala.collection.mutable.HashMap
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
+
+import spark.rdd.CoalescedRDD
 import SparkContext._
 
 class RDDSuite extends FunSuite with BeforeAndAfter {
diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala
index 90760b8a853fd720f5a2c76c67c900a3fee9d802..068607824b53389dc0dcf1deff81aca78af111a9 100644
--- a/core/src/test/scala/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/spark/ShuffleSuite.scala
@@ -1,5 +1,7 @@
 package spark
 
+import scala.collection.mutable.ArrayBuffer
+
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
 import org.scalatest.matchers.ShouldMatchers
@@ -10,8 +12,7 @@ import org.scalacheck.Prop._
 
 import com.google.common.io.Files
 
-import scala.collection.mutable.ArrayBuffer
-
+import spark.rdd.ShuffledAggregatedRDD
 import SparkContext._
 
 class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter {