diff --git a/README.md b/README.md
index 2ddfe862a23d71a300bf0bf08bc8f0015c619620..c4170650f799a9e07b2ff757775b9b85e1a1114d 100644
--- a/README.md
+++ b/README.md
@@ -27,7 +27,7 @@ Or, for the Python API, the Python shell (`./pyspark`).
 Spark also comes with several sample programs in the `examples` directory.
 To run one of them, use `./run-example <class> <params>`. For example:
 
-    ./run-example spark.examples.SparkLR local[2]
+    ./run-example org.apache.spark.examples.SparkLR local[2]
 
 will run the Logistic Regression example locally on 2 CPUs.
 
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 74990b636120de1fff01842fe478aea9127bae5a..dc63811b76a874e7fe3851726ff0dd3c166d612f 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -19,13 +19,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-assembly</artifactId>
   <name>Spark Project Assembly</name>
   <url>http://spark-project.org/</url>
@@ -40,27 +40,27 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-core</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-bagel</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-mllib</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-repl</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming</artifactId>
       <version>${project.version}</version>
     </dependency>
@@ -121,7 +121,7 @@
       <id>hadoop2-yarn</id>
       <dependencies>
         <dependency>
-          <groupId>org.spark-project</groupId>
+          <groupId>org.apache.spark</groupId>
           <artifactId>spark-yarn</artifactId>
           <version>${project.version}</version>
         </dependency>
diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml
index 4543b52c93176182300f03b67e9da8d056d531d3..47d3fa93d07657dc2401a4f2a8365b0a83fb8edb 100644
--- a/assembly/src/main/assembly/assembly.xml
+++ b/assembly/src/main/assembly/assembly.xml
@@ -30,9 +30,9 @@
     </fileSet>
     <fileSet>
       <directory>
-        ${project.parent.basedir}/core/src/main/resources/spark/ui/static/
+        ${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/
       </directory>
-      <outputDirectory>/ui-resources/spark/ui/static</outputDirectory>
+      <outputDirectory>/ui-resources/org/apache/spark/ui/static</outputDirectory>
       <includes>
         <include>**/*</include>
       </includes>
@@ -63,10 +63,10 @@
   <dependencySets>
     <dependencySet>
       <includes>
-        <include>org.spark-project:*:jar</include>
+        <include>org.apache.spark:*:jar</include>
       </includes>
       <excludes>
-        <exclude>org.spark-project:spark-assembly:jar</exclude>
+        <exclude>org.apache.spark:spark-assembly:jar</exclude>
       </excludes>
     </dependencySet>
     <dependencySet>
@@ -77,7 +77,7 @@
       <useProjectArtifact>false</useProjectArtifact>
       <excludes>
         <exclude>org.apache.hadoop:*:jar</exclude>
-        <exclude>org.spark-project:*:jar</exclude>
+        <exclude>org.apache.spark:*:jar</exclude>
       </excludes>
     </dependencySet>
   </dependencySets>
diff --git a/bagel/pom.xml b/bagel/pom.xml
index cbcf8d123930a0e1205a2ad90fa363db492d943b..9340991377aa2da542d6deef9dbb9085c69c7c8b 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -19,13 +19,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-bagel</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Bagel</name>
@@ -33,7 +33,7 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-core</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala
similarity index 86%
rename from bagel/src/main/scala/spark/bagel/Bagel.scala
rename to bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala
index 80c8d53d2bbfe498d7d57b052bc716f3c06150cc..fec8737fcdf8f0b70cead9b530320a0547619133 100644
--- a/bagel/src/main/scala/spark/bagel/Bagel.scala
+++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala
@@ -15,32 +15,31 @@
  * limitations under the License.
  */
 
-package spark.bagel
+package org.apache.spark.bagel
 
-import spark._
-import spark.SparkContext._
+import org.apache.spark._
+import org.apache.spark.SparkContext._
 
-import scala.collection.mutable.ArrayBuffer
-import storage.StorageLevel
+import org.apache.spark.storage.StorageLevel
 
 object Bagel extends Logging {
   val DEFAULT_STORAGE_LEVEL = StorageLevel.MEMORY_AND_DISK
 
   /**
    * Runs a Bagel program.
-   * @param sc [[spark.SparkContext]] to use for the program.
+   * @param sc [[org.apache.spark.SparkContext]] to use for the program.
    * @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the Key will be
    *                 the vertex id.
    * @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often this will be an
    *                 empty array, i.e. sc.parallelize(Array[K, Message]()).
-   * @param combiner [[spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one
+   * @param combiner [[org.apache.spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one
    *                message before sending (which often involves network I/O).
-   * @param aggregator [[spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep,
+   * @param aggregator [[org.apache.spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep,
    *                  and provides the result to each vertex in the next superstep.
-   * @param partitioner [[spark.Partitioner]] partitions values by key
+   * @param partitioner [[org.apache.spark.Partitioner]] partitions values by key
    * @param numPartitions number of partitions across which to split the graph.
    *                      Default is the default parallelism of the SparkContext
-   * @param storageLevel [[spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep.
+   * @param storageLevel [[org.apache.spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep.
    *                    Defaults to caching in memory.
    * @param compute function that takes a Vertex, optional set of (possibly combined) messages to the Vertex,
    *                optional Aggregator and the current superstep,
@@ -98,7 +97,7 @@ object Bagel extends Logging {
     verts
   }
 
-  /** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default storage level */
+  /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default storage level */
   def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
     sc: SparkContext,
     vertices: RDD[(K, V)],
@@ -110,7 +109,7 @@ object Bagel extends Logging {
     compute: (V, Option[C], Int) => (V, Array[M])
   ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)
 
-  /** Runs a Bagel program with no [[spark.bagel.Aggregator]] */
+  /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] */
   def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
     sc: SparkContext,
     vertices: RDD[(K, V)],
@@ -128,7 +127,7 @@ object Bagel extends Logging {
   }
 
   /**
-   * Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]]
+   * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]]
    * and default storage level
    */
   def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
@@ -141,7 +140,7 @@ object Bagel extends Logging {
     compute: (V, Option[C], Int) => (V, Array[M])
   ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)
 
-  /** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default [[spark.HashPartitioner]]*/
+  /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default [[org.apache.spark.HashPartitioner]]*/
   def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
     sc: SparkContext,
     vertices: RDD[(K, V)],
@@ -159,8 +158,8 @@ object Bagel extends Logging {
   }
 
   /**
-   * Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]],
-   * [[spark.bagel.DefaultCombiner]] and the default storage level
+   * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]],
+   * [[org.apache.spark.bagel.DefaultCombiner]] and the default storage level
    */
   def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
     sc: SparkContext,
@@ -172,8 +171,8 @@ object Bagel extends Logging {
   ): RDD[(K, V)] = run(sc, vertices, messages, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)
 
   /**
-   * Runs a Bagel program with no [[spark.bagel.Aggregator]], the default [[spark.HashPartitioner]]
-   * and [[spark.bagel.DefaultCombiner]]
+   * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], the default [[org.apache.spark.HashPartitioner]]
+   * and [[org.apache.spark.bagel.DefaultCombiner]]
    */
   def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
     sc: SparkContext,
diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala
similarity index 96%
rename from bagel/src/test/scala/bagel/BagelSuite.scala
rename to bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala
index ef2d57fbd0a81c18f9777c8b90e16b54c4569c0c..7b954a477570f5c42e0b8ee4529b7234f751667e 100644
--- a/bagel/src/test/scala/bagel/BagelSuite.scala
+++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala
@@ -15,16 +15,14 @@
  * limitations under the License.
  */
 
-package spark.bagel
+package org.apache.spark.bagel
 
-import org.scalatest.{FunSuite, Assertions, BeforeAndAfter}
+import org.scalatest.{BeforeAndAfter, FunSuite, Assertions}
 import org.scalatest.concurrent.Timeouts
 import org.scalatest.time.SpanSugar._
 
-import scala.collection.mutable.ArrayBuffer
-
-import spark._
-import storage.StorageLevel
+import org.apache.spark._
+import org.apache.spark.storage.StorageLevel
 
 class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable
 class TestMessage(val targetId: String) extends Message[String] with Serializable
diff --git a/bin/start-master.sh b/bin/start-master.sh
index 2288fb19d7b0d022dbd603a4c26b61de0c9d4648..648c7ae75fe81ddafbd9394ee6f1f60f00f7c263 100755
--- a/bin/start-master.sh
+++ b/bin/start-master.sh
@@ -49,4 +49,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
     fi
 fi
 
-"$bin"/spark-daemon.sh start spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
+"$bin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
diff --git a/bin/start-slave.sh b/bin/start-slave.sh
index d6db16882d0206a5a58385188b53f55b7f365e7b..4eefa209443385881905ee0c7208ea56fa79e9e5 100755
--- a/bin/start-slave.sh
+++ b/bin/start-slave.sh
@@ -32,4 +32,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
     fi
 fi
 
-"$bin"/spark-daemon.sh start spark.deploy.worker.Worker "$@"
+"$bin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"
diff --git a/bin/stop-master.sh b/bin/stop-master.sh
index 31a610bf9df12c9f504194934e2d60d4633b34b5..310e33bedc057194d4b410f2335234f5e2f30613 100755
--- a/bin/stop-master.sh
+++ b/bin/stop-master.sh
@@ -24,4 +24,4 @@ bin=`cd "$bin"; pwd`
 
 . "$bin/spark-config.sh"
 
-"$bin"/spark-daemon.sh stop spark.deploy.master.Master 1
+"$bin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1
diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh
index 8e056f23d4ee1757bd335ef942961cc64cd7a02d..03e416a13274d5c94caa381257332ec42bb31fd1 100755
--- a/bin/stop-slaves.sh
+++ b/bin/stop-slaves.sh
@@ -29,9 +29,9 @@ if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
 fi
 
 if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
-  "$bin"/spark-daemons.sh stop spark.deploy.worker.Worker 1
+  "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
 else
   for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
-    "$bin"/spark-daemons.sh stop spark.deploy.worker.Worker $(( $i + 1 ))
+    "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
   done
 fi
diff --git a/core/pom.xml b/core/pom.xml
index 53696367e984f88682071abf03001337f76a92f2..c803217f96418900df585960347f0152330f1ab1 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -19,13 +19,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-core</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Core</name>
diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java
similarity index 98%
rename from core/src/main/java/spark/network/netty/FileClient.java
rename to core/src/main/java/org/apache/spark/network/netty/FileClient.java
index 0625a6d502944dde21949c414b38fd71e65af037..20a7a3aa8c122ba7696df735bb3309c21fd50691 100644
--- a/core/src/main/java/spark/network/netty/FileClient.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network.netty;
+package org.apache.spark.network.netty;
 
 import io.netty.bootstrap.Bootstrap;
 import io.netty.channel.Channel;
diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java
similarity index 97%
rename from core/src/main/java/spark/network/netty/FileClientChannelInitializer.java
rename to core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java
index 05ad4b61d72919cefb89f98490355c2c47331bd3..65ee15d63b8548b27275419a2c1507cf0e23847b 100644
--- a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network.netty;
+package org.apache.spark.network.netty;
 
 import io.netty.buffer.BufType;
 import io.netty.channel.ChannelInitializer;
diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
similarity index 97%
rename from core/src/main/java/spark/network/netty/FileClientHandler.java
rename to core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
index e8cd9801f63eb56d68be6700db378d582bffb11e..c4aa2669e05d36bea4bd6631ef5c96bd63634219 100644
--- a/core/src/main/java/spark/network/netty/FileClientHandler.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network.netty;
+package org.apache.spark.network.netty;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelHandlerContext;
diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java
similarity index 98%
rename from core/src/main/java/spark/network/netty/FileServer.java
rename to core/src/main/java/org/apache/spark/network/netty/FileServer.java
index 9f009a61d5dfbc3dbb83fdaa299c3140ee9e61d5..666432474dc75616a3f3a586c55996f5acdc8635 100644
--- a/core/src/main/java/spark/network/netty/FileServer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network.netty;
+package org.apache.spark.network.netty;
 
 import java.net.InetSocketAddress;
 
diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java
similarity index 97%
rename from core/src/main/java/spark/network/netty/FileServerChannelInitializer.java
rename to core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java
index 50c57a81a355313e741f7abed9a09f32048960d5..833af1632de9ca2b87d0b51f5732ae359c4e2d75 100644
--- a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network.netty;
+package org.apache.spark.network.netty;
 
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.socket.SocketChannel;
diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
similarity index 98%
rename from core/src/main/java/spark/network/netty/FileServerHandler.java
rename to core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
index 176ba8da49102ffa0d54b3c8d78cf6b1d6c602c8..d3d57a02555720000bb90583bfde2ab4f32d07d1 100644
--- a/core/src/main/java/spark/network/netty/FileServerHandler.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network.netty;
+package org.apache.spark.network.netty;
 
 import java.io.File;
 import java.io.FileInputStream;
diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
similarity index 93%
rename from core/src/main/java/spark/network/netty/PathResolver.java
rename to core/src/main/java/org/apache/spark/network/netty/PathResolver.java
index f446c55b19b6b922ac1bcc7a29e6a6cbbe2399d9..94c034cad0119397bc6332dde066fa2be2ab75a8 100755
--- a/core/src/main/java/spark/network/netty/PathResolver.java
+++ b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network.netty;
+package org.apache.spark.network.netty;
 
 
 public interface PathResolver {
diff --git a/core/src/main/resources/spark/ui/static/bootstrap.min.css b/core/src/main/resources/org/apache/spark/ui/static/bootstrap.min.css
similarity index 100%
rename from core/src/main/resources/spark/ui/static/bootstrap.min.css
rename to core/src/main/resources/org/apache/spark/ui/static/bootstrap.min.css
diff --git a/core/src/main/resources/spark/ui/static/sorttable.js b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js
similarity index 100%
rename from core/src/main/resources/spark/ui/static/sorttable.js
rename to core/src/main/resources/org/apache/spark/ui/static/sorttable.js
diff --git a/core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png b/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png
similarity index 100%
rename from core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png
rename to core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png
diff --git a/core/src/main/resources/spark/ui/static/spark_logo.png b/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png
similarity index 100%
rename from core/src/main/resources/spark/ui/static/spark_logo.png
rename to core/src/main/resources/org/apache/spark/ui/static/spark_logo.png
diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css
similarity index 100%
rename from core/src/main/resources/spark/ui/static/webui.css
rename to core/src/main/resources/org/apache/spark/ui/static/webui.css
diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala
similarity index 95%
rename from core/src/main/scala/spark/Accumulators.scala
rename to core/src/main/scala/org/apache/spark/Accumulators.scala
index 6ff92ce833b022778a5f8d72bcdb6abb9167be61..5177ee58faa482c0cbfada60b0e3d3dc908ee50a 100644
--- a/core/src/main/scala/spark/Accumulators.scala
+++ b/core/src/main/scala/org/apache/spark/Accumulators.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io._
 
@@ -28,7 +28,7 @@ import scala.collection.generic.Growable
  *
  * You must define how to add data, and how to merge two of these together.  For some datatypes,
  * such as a counter, these might be the same operation. In that case, you can use the simpler
- * [[spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are
+ * [[org.apache.spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are
  * accumulating a set. You will add items to the set, and you will union two sets together.
  *
  * @param initialValue initial value of accumulator
@@ -176,7 +176,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser
   def zero(initialValue: R): R = {
     // We need to clone initialValue, but it's hard to specify that R should also be Cloneable.
     // Instead we'll serialize it to a buffer and load it back.
-    val ser = (new spark.JavaSerializer).newInstance()
+    val ser = new JavaSerializer().newInstance()
     val copy = ser.deserialize[R](ser.serialize(initialValue))
     copy.clear()   // In case it contained stuff
     copy
@@ -184,7 +184,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser
 }
 
 /**
- * A simpler value of [[spark.Accumulable]] where the result type being accumulated is the same
+ * A simpler value of [[org.apache.spark.Accumulable]] where the result type being accumulated is the same
  * as the types of elements being merged.
  *
  * @param initialValue initial value of accumulator
@@ -195,7 +195,7 @@ class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T])
   extends Accumulable[T,T](initialValue, param)
 
 /**
- * A simpler version of [[spark.AccumulableParam]] where the only datatype you can add in is the same type
+ * A simpler version of [[org.apache.spark.AccumulableParam]] where the only datatype you can add in is the same type
  * as the accumulated value. An implicit AccumulatorParam object needs to be available when you create
  * Accumulators of a specific type.
  *
diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
similarity index 98%
rename from core/src/main/scala/spark/Aggregator.scala
rename to core/src/main/scala/org/apache/spark/Aggregator.scala
index 9af401986d350ec4b47495638d64ccc33531034a..3ef402926e8985e30744da8f0e67a30715b21c4e 100644
--- a/core/src/main/scala/spark/Aggregator.scala
+++ b/core/src/main/scala/org/apache/spark/Aggregator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.util.{HashMap => JHashMap}
 
diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala
similarity index 93%
rename from core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
rename to core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala
index 1ec95ed9b88e049b72d30d56f06f0c386fbc6ebc..908ff56a6bbdb8bcd00a6ab671ee298ea8c67ed8 100644
--- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
+++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 
-import spark.executor.{ShuffleReadMetrics, TaskMetrics}
-import spark.serializer.Serializer
-import spark.storage.BlockManagerId
-import spark.util.CompletionIterator
+import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.util.CompletionIterator
 
 
 private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala
similarity index 97%
rename from core/src/main/scala/spark/CacheManager.scala
rename to core/src/main/scala/org/apache/spark/CacheManager.scala
index 81314805a93853af4855d429c883c2e216288034..42e465b9d8631481e0f48126d917a2c631285bee 100644
--- a/core/src/main/scala/spark/CacheManager.scala
+++ b/core/src/main/scala/org/apache/spark/CacheManager.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import scala.collection.mutable.{ArrayBuffer, HashSet}
-import spark.storage.{BlockManager, StorageLevel}
+import org.apache.spark.storage.{BlockManager, StorageLevel}
 
 
 /** Spark class responsible for passing RDDs split contents to the BlockManager and making
diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/ClosureCleaner.scala
similarity index 99%
rename from core/src/main/scala/spark/ClosureCleaner.scala
rename to core/src/main/scala/org/apache/spark/ClosureCleaner.scala
index 8b39241095b886e8ecb0d6831a33a99ec245e3a3..71d9e62d4f0d60330ff1bf5b241b604b51642228 100644
--- a/core/src/main/scala/spark/ClosureCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/ClosureCleaner.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.lang.reflect.Field
 
diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala
similarity index 99%
rename from core/src/main/scala/spark/Dependency.scala
rename to core/src/main/scala/org/apache/spark/Dependency.scala
index d5a960657095634151ddb3e4513b934212a0e408..cc3c2474a6d80d040439955171918df55368e224 100644
--- a/core/src/main/scala/spark/Dependency.scala
+++ b/core/src/main/scala/org/apache/spark/Dependency.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 /**
  * Base class for dependencies.
diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala
similarity index 89%
rename from core/src/main/scala/spark/DoubleRDDFunctions.scala
rename to core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala
index 104168e61ccdfbf70e5d504b18481decfc2d41b6..dd344491b833c3f105393267edfaa2b27f2a59a7 100644
--- a/core/src/main/scala/spark/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
-import spark.partial.BoundedDouble
-import spark.partial.MeanEvaluator
-import spark.partial.PartialResult
-import spark.partial.SumEvaluator
-import spark.util.StatCounter
+import org.apache.spark.partial.BoundedDouble
+import org.apache.spark.partial.MeanEvaluator
+import org.apache.spark.partial.PartialResult
+import org.apache.spark.partial.SumEvaluator
+import org.apache.spark.util.StatCounter
 
 /**
  * Extra functions available on RDDs of Doubles through an implicit conversion.
@@ -34,7 +34,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
   }
 
   /**
-   * Return a [[spark.util.StatCounter]] object that captures the mean, variance and count
+   * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and count
    * of the RDD's elements in one operation.
    */
   def stats(): StatCounter = {
diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/FetchFailedException.scala
similarity index 95%
rename from core/src/main/scala/spark/FetchFailedException.scala
rename to core/src/main/scala/org/apache/spark/FetchFailedException.scala
index a2dae6cae9adee2370f297a6eef3f4da6e857865..d242047502fd3acb320264533ef88a4443c4fe3b 100644
--- a/core/src/main/scala/spark/FetchFailedException.scala
+++ b/core/src/main/scala/org/apache/spark/FetchFailedException.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
-import spark.storage.BlockManagerId
+import org.apache.spark.storage.BlockManagerId
 
 private[spark] class FetchFailedException(
     taskEndReason: TaskEndReason,
diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala
similarity index 98%
rename from core/src/main/scala/spark/HttpFileServer.scala
rename to core/src/main/scala/org/apache/spark/HttpFileServer.scala
index a13a7a28590bbc668099c9347e38c43847a9fa33..9b3a896648943c58e391b65f79a648f01e984ccd 100644
--- a/core/src/main/scala/spark/HttpFileServer.scala
+++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io.{File}
 import com.google.common.io.Files
diff --git a/core/src/main/scala/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala
similarity index 99%
rename from core/src/main/scala/spark/HttpServer.scala
rename to core/src/main/scala/org/apache/spark/HttpServer.scala
index c9dffbc63110b2c5b780d2923b24ea11610f9d81..db36c7c9dd12b8e758254b745375c95e73957dda 100644
--- a/core/src/main/scala/spark/HttpServer.scala
+++ b/core/src/main/scala/org/apache/spark/HttpServer.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io.File
 import java.net.InetAddress
diff --git a/core/src/main/scala/spark/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/JavaSerializer.scala
similarity index 97%
rename from core/src/main/scala/spark/JavaSerializer.scala
rename to core/src/main/scala/org/apache/spark/JavaSerializer.scala
index 04c5f44e6be363c0a2ab9bf0b66297ada36437f6..f43396cb6b04efb8c700a00e57446371296ea5e0 100644
--- a/core/src/main/scala/spark/JavaSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/JavaSerializer.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io._
 import java.nio.ByteBuffer
 
 import serializer.{Serializer, SerializerInstance, DeserializationStream, SerializationStream}
-import spark.util.ByteBufferInputStream
+import org.apache.spark.util.ByteBufferInputStream
 
 private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream {
   val objOut = new ObjectOutputStream(out)
diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/KryoSerializer.scala
similarity index 96%
rename from core/src/main/scala/spark/KryoSerializer.scala
rename to core/src/main/scala/org/apache/spark/KryoSerializer.scala
index eeb2993d8a5081bdecc66bb6049fb19d41dc39e3..db86e6db439382ccd0f816a6eb24986e3515b7b7 100644
--- a/core/src/main/scala/spark/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/KryoSerializer.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io._
 import java.nio.ByteBuffer
@@ -24,8 +24,8 @@ import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
 import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer}
 import com.twitter.chill.ScalaKryoInstantiator
 import serializer.{SerializerInstance, DeserializationStream, SerializationStream}
-import spark.broadcast._
-import spark.storage._
+import org.apache.spark.broadcast._
+import org.apache.spark.storage._
 
 private[spark]
 class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream {
@@ -104,7 +104,7 @@ trait KryoRegistrator {
 /**
  * A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]].
  */
-class KryoSerializer extends spark.serializer.Serializer with Logging {
+class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging {
   private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
 
   def newKryoOutput() = new KryoOutput(bufferSize)
@@ -153,4 +153,4 @@ class KryoSerializer extends spark.serializer.Serializer with Logging {
   def newInstance(): SerializerInstance = {
     new KryoSerializerInstance(this)
   }
-}
\ No newline at end of file
+}
diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
similarity index 99%
rename from core/src/main/scala/spark/Logging.scala
rename to core/src/main/scala/org/apache/spark/Logging.scala
index 79b0362830d9d43550e81f67679774dcccda17be..6a973ea4951c3a389a202f6c5dfd77f355061648 100644
--- a/core/src/main/scala/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
similarity index 98%
rename from core/src/main/scala/spark/MapOutputTracker.scala
rename to core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 0cd0341a7242e18006e5dbfa0e0235472222a12a..0f422d910a19a9607577641e21f89e1b2a53d98e 100644
--- a/core/src/main/scala/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io._
 import java.util.zip.{GZIPInputStream, GZIPOutputStream}
@@ -30,9 +30,9 @@ import akka.remote._
 import akka.util.Duration
 
 
-import spark.scheduler.MapStatus
-import spark.storage.BlockManagerId
-import spark.util.{MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.scheduler.MapStatus
+import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
 
 
 private[spark] sealed trait MapOutputTrackerMessage
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/PairRDDFunctions.scala
similarity index 98%
rename from core/src/main/scala/spark/PairRDDFunctions.scala
rename to core/src/main/scala/org/apache/spark/PairRDDFunctions.scala
index cc1285dd95f4765d9cbcd994db021cb1830031d7..d046e7c1a4f3cfb6093a0d091726ae87fa0862f7 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/PairRDDFunctions.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.nio.ByteBuffer
 import java.util.{Date, HashMap => JHashMap}
@@ -40,11 +40,11 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat,
     RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, SparkHadoopMapReduceUtil}
 import org.apache.hadoop.security.UserGroupInformation
 
-import spark.partial.BoundedDouble
-import spark.partial.PartialResult
-import spark.rdd._
-import spark.SparkContext._
-import spark.Partitioner._
+import org.apache.spark.partial.BoundedDouble
+import org.apache.spark.partial.PartialResult
+import org.apache.spark.rdd._
+import org.apache.spark.SparkContext._
+import org.apache.spark.Partitioner._
 
 /**
  * Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
@@ -559,7 +559,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
     val formatter = new SimpleDateFormat("yyyyMMddHHmm")
     val jobtrackerID = formatter.format(new Date())
     val stageId = self.id
-    def writeShard(context: spark.TaskContext, iter: Iterator[(K,V)]): Int = {
+    def writeShard(context: TaskContext, iter: Iterator[(K,V)]): Int = {
       // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it
       // around by taking a mod. We expect that no task will be attempted 2 billion times.
       val attemptNumber = (context.attemptId % Int.MaxValue).toInt
@@ -571,7 +571,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
       committer.setupTask(hadoopContext)
       val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]]
       while (iter.hasNext) {
-        val (k, v) = iter.next
+        val (k, v) = iter.next()
         writer.write(k, v)
       }
       writer.close(hadoopContext)
diff --git a/core/src/main/scala/spark/Partition.scala b/core/src/main/scala/org/apache/spark/Partition.scala
similarity index 97%
rename from core/src/main/scala/spark/Partition.scala
rename to core/src/main/scala/org/apache/spark/Partition.scala
index 2a4edcec9841948ff0b3cca294b5a0f14aca786c..87914a061f5d7c0864b2a2f2e0abedc260be3831 100644
--- a/core/src/main/scala/spark/Partition.scala
+++ b/core/src/main/scala/org/apache/spark/Partition.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 /**
  * A partition of an RDD.
diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
similarity index 94%
rename from core/src/main/scala/spark/Partitioner.scala
rename to core/src/main/scala/org/apache/spark/Partitioner.scala
index 65da8235d75085bdca8ce20f6c40a537990fd203..4dce2607b0a1afbecc35a19fb00d4a1e89fe2a6e 100644
--- a/core/src/main/scala/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 /**
  * An object that defines how the elements in a key-value pair RDD are partitioned by key.
@@ -56,7 +56,7 @@ object Partitioner {
 }
 
 /**
- * A [[spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`.
+ * A [[org.apache.spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`.
  *
  * Java arrays have hashCodes that are based on the arrays' identities rather than their contents,
  * so attempting to partition an RDD[Array[_]] or RDD[(Array[_], _)] using a HashPartitioner will
@@ -79,7 +79,7 @@ class HashPartitioner(partitions: Int) extends Partitioner {
 }
 
 /**
- * A [[spark.Partitioner]] that partitions sortable records by range into roughly equal ranges.
+ * A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly equal ranges.
  * Determines the ranges by sampling the RDD passed in.
  */
 class RangePartitioner[K <% Ordered[K]: ClassManifest, V](
diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/org/apache/spark/RDD.scala
similarity index 95%
rename from core/src/main/scala/spark/RDD.scala
rename to core/src/main/scala/org/apache/spark/RDD.scala
index 25a6951732b055d87123be8b6044ca0eb548bd98..0d1f07f76cb00b020198822262bd47092f920114 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/RDD.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.util.Random
 
@@ -31,30 +31,30 @@ import org.apache.hadoop.mapred.TextOutputFormat
 
 import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
 
-import spark.Partitioner._
-import spark.api.java.JavaRDD
-import spark.partial.BoundedDouble
-import spark.partial.CountEvaluator
-import spark.partial.GroupedCountEvaluator
-import spark.partial.PartialResult
-import spark.rdd.CoalescedRDD
-import spark.rdd.CartesianRDD
-import spark.rdd.FilteredRDD
-import spark.rdd.FlatMappedRDD
-import spark.rdd.GlommedRDD
-import spark.rdd.MappedRDD
-import spark.rdd.MapPartitionsRDD
-import spark.rdd.MapPartitionsWithIndexRDD
-import spark.rdd.PipedRDD
-import spark.rdd.SampledRDD
-import spark.rdd.ShuffledRDD
-import spark.rdd.UnionRDD
-import spark.rdd.ZippedRDD
-import spark.rdd.ZippedPartitionsRDD2
-import spark.rdd.ZippedPartitionsRDD3
-import spark.rdd.ZippedPartitionsRDD4
-import spark.storage.StorageLevel
-import spark.util.BoundedPriorityQueue
+import org.apache.spark.Partitioner._
+import org.apache.spark.api.java.JavaRDD
+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.rdd.CoalescedRDD
+import org.apache.spark.rdd.CartesianRDD
+import org.apache.spark.rdd.FilteredRDD
+import org.apache.spark.rdd.FlatMappedRDD
+import org.apache.spark.rdd.GlommedRDD
+import org.apache.spark.rdd.MappedRDD
+import org.apache.spark.rdd.MapPartitionsRDD
+import org.apache.spark.rdd.MapPartitionsWithIndexRDD
+import org.apache.spark.rdd.PipedRDD
+import org.apache.spark.rdd.SampledRDD
+import org.apache.spark.rdd.ShuffledRDD
+import org.apache.spark.rdd.UnionRDD
+import org.apache.spark.rdd.ZippedRDD
+import org.apache.spark.rdd.ZippedPartitionsRDD2
+import org.apache.spark.rdd.ZippedPartitionsRDD3
+import org.apache.spark.rdd.ZippedPartitionsRDD4
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.BoundedPriorityQueue
 
 import SparkContext._
 
@@ -62,12 +62,12 @@ import SparkContext._
  * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable,
  * partitioned collection of elements that can be operated on in parallel. This class contains the
  * basic operations available on all RDDs, such as `map`, `filter`, and `persist`. In addition,
- * [[spark.PairRDDFunctions]] contains operations available only on RDDs of key-value pairs, such
- * as `groupByKey` and `join`; [[spark.DoubleRDDFunctions]] contains operations available only on
- * RDDs of Doubles; and [[spark.SequenceFileRDDFunctions]] contains operations available on RDDs
+ * [[org.apache.spark.PairRDDFunctions]] contains operations available only on RDDs of key-value pairs, such
+ * as `groupByKey` and `join`; [[org.apache.spark.DoubleRDDFunctions]] contains operations available only on
+ * RDDs of Doubles; and [[org.apache.spark.SequenceFileRDDFunctions]] contains operations available on RDDs
  * that can be saved as SequenceFiles. These operations are automatically available on any RDD of
  * the right type (e.g. RDD[(Int, Int)] through implicit conversions when you
- * `import spark.SparkContext._`.
+ * `import org.apache.spark.SparkContext._`.
  *
  * Internally, each RDD is characterized by five main properties:
  *
@@ -893,7 +893,7 @@ abstract class RDD[T: ClassManifest](
     dependencies.head.rdd.asInstanceOf[RDD[U]]
   }
 
-  /** The [[spark.SparkContext]] that this RDD was created on. */
+  /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */
   def context = sc
 
   // Avoid handling doCheckpoint multiple times to prevent excessive recursion
@@ -929,7 +929,7 @@ abstract class RDD[T: ClassManifest](
    * Clears the dependencies of this RDD. This method must ensure that all references
    * to the original parent RDDs is removed to enable the parent RDDs to be garbage
    * collected. Subclasses of RDD may override this method for implementing their own cleaning
-   * logic. See [[spark.rdd.UnionRDD]] for an example.
+   * logic. See [[org.apache.spark.rdd.UnionRDD]] for an example.
    */
   protected def clearDependencies() {
     dependencies_ = null
diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/RDDCheckpointData.scala
similarity index 99%
rename from core/src/main/scala/spark/RDDCheckpointData.scala
rename to core/src/main/scala/org/apache/spark/RDDCheckpointData.scala
index b615f820eb47ac7dd3a8e7d45a5c112c71cbd389..0334de6924c267420160b37775dcad0fc69a9185 100644
--- a/core/src/main/scala/spark/RDDCheckpointData.scala
+++ b/core/src/main/scala/org/apache/spark/RDDCheckpointData.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.conf.Configuration
diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala
similarity index 98%
rename from core/src/main/scala/spark/SequenceFileRDDFunctions.scala
rename to core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala
index 9f30b7f22f2b8e72110bdbded9b92df17240f79d..d58fb4e4bcb20f48711a0adf636dc4cdfdb8abff 100644
--- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io.EOFException
 import java.net.URL
@@ -41,7 +41,7 @@ import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.io.BytesWritable
 import org.apache.hadoop.io.Text
 
-import spark.SparkContext._
+import org.apache.spark.SparkContext._
 
 /**
  * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile,
diff --git a/core/src/main/scala/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala
similarity index 98%
rename from core/src/main/scala/spark/SerializableWritable.scala
rename to core/src/main/scala/org/apache/spark/SerializableWritable.scala
index 936d8e62412e7853ed823e66f53fd95a13811c6a..fdd4c24e2345f7bff20a1e565d9d50ee60c47e0f 100644
--- a/core/src/main/scala/spark/SerializableWritable.scala
+++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io._
 
diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala
similarity index 91%
rename from core/src/main/scala/spark/ShuffleFetcher.scala
rename to core/src/main/scala/org/apache/spark/ShuffleFetcher.scala
index a6839cf7a40dc94b6cd49010acee7f8d259c5f5c..307c383a89c72d1af1670b4176551719b226ba2e 100644
--- a/core/src/main/scala/spark/ShuffleFetcher.scala
+++ b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
-import spark.executor.TaskMetrics
-import spark.serializer.Serializer
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.serializer.Serializer
 
 
 private[spark] abstract class ShuffleFetcher {
diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/SizeEstimator.scala
similarity index 99%
rename from core/src/main/scala/spark/SizeEstimator.scala
rename to core/src/main/scala/org/apache/spark/SizeEstimator.scala
index 6cc57566d79c3023c63a1bbde4917f7c8f58ce28..4bfc837710307a582fc5cbe970498e4ae2e5c3fd 100644
--- a/core/src/main/scala/spark/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/SizeEstimator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.lang.reflect.Field
 import java.lang.reflect.Modifier
diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
similarity index 97%
rename from core/src/main/scala/spark/SparkContext.scala
rename to core/src/main/scala/org/apache/spark/SparkContext.scala
index 7ce9505b9c277966a5eb73b41a28d585a4413445..1207b242bc04566fd165bc0c8cec2a9830cc6d8e 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io._
 import java.net.URI
@@ -52,22 +52,22 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor
 
 import org.apache.mesos.MesosNativeLibrary
 
-import spark.deploy.LocalSparkCluster
-import spark.partial.{ApproximateEvaluator, PartialResult}
-import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD,
+import org.apache.spark.deploy.LocalSparkCluster
+import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
+import org.apache.spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD,
   OrderedRDDFunctions}
-import spark.scheduler._
-import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend,
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend,
   ClusterScheduler, Schedulable, SchedulingMode}
-import spark.scheduler.local.LocalScheduler
-import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
-import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource}
-import spark.ui.SparkUI
-import spark.util.{MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.scheduler.local.LocalScheduler
+import org.apache.spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
+import org.apache.spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource}
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
 import scala.Some
-import spark.scheduler.StageInfo
-import spark.storage.RDDInfo
-import spark.storage.StorageStatus
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.storage.RDDInfo
+import org.apache.spark.storage.StorageStatus
 
 /**
  * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
@@ -494,14 +494,14 @@ class SparkContext(
   // Methods for creating shared variables
 
   /**
-   * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values
+   * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values
    * to using the `+=` method. Only the driver can access the accumulator's `value`.
    */
   def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) =
     new Accumulator(initialValue, param)
 
   /**
-   * Create an [[spark.Accumulable]] shared variable, to which tasks can add values with `+=`.
+   * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values with `+=`.
    * Only the driver can access the accumuable's `value`.
    * @tparam T accumulator type
    * @tparam R type that can be added to the accumulator
@@ -521,7 +521,7 @@ class SparkContext(
   }
 
   /**
-   * Broadcast a read-only variable to the cluster, returning a [[spark.broadcast.Broadcast]] object for
+   * Broadcast a read-only variable to the cluster, returning a [[org.apache.spark.broadcast.Broadcast]] object for
    * reading it in distributed functions. The variable will be sent to each cluster only once.
    */
   def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal)
diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
similarity index 91%
rename from core/src/main/scala/spark/SparkEnv.scala
rename to core/src/main/scala/org/apache/spark/SparkEnv.scala
index 1f66e9cc7f3b36a59d054f7ab0aa453e16886544..6e6fe5df6b66a165c9774aa01a7cab136fce784f 100644
--- a/core/src/main/scala/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import collection.mutable
 import serializer.Serializer
@@ -23,15 +23,14 @@ import serializer.Serializer
 import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem}
 import akka.remote.RemoteActorRefProvider
 
-import spark.broadcast.BroadcastManager
-import spark.metrics.MetricsSystem
-import spark.deploy.SparkHadoopUtil
-import spark.storage.BlockManager
-import spark.storage.BlockManagerMaster
-import spark.network.ConnectionManager
-import spark.serializer.{Serializer, SerializerManager}
-import spark.util.AkkaUtils
-import spark.api.python.PythonWorkerFactory
+import org.apache.spark.broadcast.BroadcastManager
+import org.apache.spark.metrics.MetricsSystem
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.storage.{BlockManagerMasterActor, BlockManager, BlockManagerMaster}
+import org.apache.spark.network.ConnectionManager
+import org.apache.spark.serializer.{Serializer, SerializerManager}
+import org.apache.spark.util.AkkaUtils
+import org.apache.spark.api.python.PythonWorkerFactory
 
 
 /**
@@ -156,10 +155,10 @@ object SparkEnv extends Logging {
     val serializerManager = new SerializerManager
 
     val serializer = serializerManager.setDefault(
-      System.getProperty("spark.serializer", "spark.JavaSerializer"))
+      System.getProperty("spark.serializer", "org.apache.spark.JavaSerializer"))
 
     val closureSerializer = serializerManager.get(
-      System.getProperty("spark.closure.serializer", "spark.JavaSerializer"))
+      System.getProperty("spark.closure.serializer", "org.apache.spark.JavaSerializer"))
 
     def registerOrLookup(name: String, newActor: => Actor): ActorRef = {
       if (isDriver) {
@@ -177,7 +176,7 @@ object SparkEnv extends Logging {
 
     val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
       "BlockManagerMaster",
-      new spark.storage.BlockManagerMasterActor(isLocal)))
+      new BlockManagerMasterActor(isLocal)))
     val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer)
 
     val connectionManager = blockManager.connectionManager
@@ -194,7 +193,7 @@ object SparkEnv extends Logging {
       new MapOutputTrackerActor(mapOutputTracker))
 
     val shuffleFetcher = instantiateClass[ShuffleFetcher](
-      "spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher")
+      "spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher")
 
     val httpFileServer = new HttpFileServer()
     httpFileServer.initialize()
diff --git a/core/src/main/scala/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala
similarity index 97%
rename from core/src/main/scala/spark/SparkException.scala
rename to core/src/main/scala/org/apache/spark/SparkException.scala
index b7045eea63e86467f58ec1cfc05a904fa08e805b..d34e47e8cac226ef63f82ae5dc1ad023b4be1bae 100644
--- a/core/src/main/scala/spark/SparkException.scala
+++ b/core/src/main/scala/org/apache/spark/SparkException.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 class SparkException(message: String, cause: Throwable)
   extends Exception(message, cause) {
diff --git a/core/src/main/scala/spark/SparkFiles.java b/core/src/main/scala/org/apache/spark/SparkFiles.java
similarity index 98%
rename from core/src/main/scala/spark/SparkFiles.java
rename to core/src/main/scala/org/apache/spark/SparkFiles.java
index f9b3f7965eaf1529bcee091aab8f430209eef8c4..af9cf85e372bf7b8f22508690552e7e04de8229d 100644
--- a/core/src/main/scala/spark/SparkFiles.java
+++ b/core/src/main/scala/org/apache/spark/SparkFiles.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark;
+package org.apache.spark;
 
 import java.io.File;
 
diff --git a/core/src/main/scala/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
similarity index 98%
rename from core/src/main/scala/spark/SparkHadoopWriter.scala
rename to core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
index 6b330ef5727be767df763dd61c42caa88012b353..2bab9d6e3d9280c2173fa2b6f2498c32d03bb6cc 100644
--- a/core/src/main/scala/spark/SparkHadoopWriter.scala
+++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
@@ -25,8 +25,8 @@ import java.text.NumberFormat
 import java.io.IOException
 import java.util.Date
 
-import spark.Logging
-import spark.SerializableWritable
+import org.apache.spark.Logging
+import org.apache.spark.SerializableWritable
 
 /**
  * Internal helper class that saves an RDD using a Hadoop OutputFormat. This is only public
diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
similarity index 98%
rename from core/src/main/scala/spark/TaskContext.scala
rename to core/src/main/scala/org/apache/spark/TaskContext.scala
index b79f4ca81306837d01206d168405ddc22b84e495..b2dd668330a3acdc68a0f84160b09a1bcbbcd073 100644
--- a/core/src/main/scala/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import executor.TaskMetrics
 import scala.collection.mutable.ArrayBuffer
diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
similarity index 93%
rename from core/src/main/scala/spark/TaskEndReason.scala
rename to core/src/main/scala/org/apache/spark/TaskEndReason.scala
index 3ad665da34481677a8c5fb22de1f7742d96975e1..03bf268863cf7a9942eb8b0b3c8777aa7a79fd5f 100644
--- a/core/src/main/scala/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
-import spark.executor.TaskMetrics
-import spark.storage.BlockManagerId
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.storage.BlockManagerId
 
 /**
  * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry
diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala
similarity index 98%
rename from core/src/main/scala/spark/TaskState.scala
rename to core/src/main/scala/org/apache/spark/TaskState.scala
index bf757530565db259fe40a4064c66ae7b4ed9288d..19ce8369d90c7e2bedad49625557fb08b17492c7 100644
--- a/core/src/main/scala/spark/TaskState.scala
+++ b/core/src/main/scala/org/apache/spark/TaskState.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.apache.mesos.Protos.{TaskState => MesosTaskState}
 
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/org/apache/spark/Utils.scala
similarity index 99%
rename from core/src/main/scala/spark/Utils.scala
rename to core/src/main/scala/org/apache/spark/Utils.scala
index bb8aad3f4ce0a316085be423152f266375fdd4fb..1e17deb010d69e8af090b548af3a5ab953411050 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/Utils.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io._
 import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket}
@@ -33,8 +33,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder
 
 import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
 
-import spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}
-import spark.deploy.SparkHadoopUtil
+import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}
+import org.apache.spark.deploy.SparkHadoopUtil
 import java.nio.ByteBuffer
 
 
diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
similarity index 92%
rename from core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
rename to core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
index 8ce7df62133876f73b60976deea9eb7481e5fc02..cb25ff728eeca78bf6dedfde40bf065a441a3cba 100644
--- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package spark.api.java
-
-import spark.RDD
-import spark.SparkContext.doubleRDDToDoubleRDDFunctions
-import spark.api.java.function.{Function => JFunction}
-import spark.util.StatCounter
-import spark.partial.{BoundedDouble, PartialResult}
-import spark.storage.StorageLevel
+package org.apache.spark.api.java
+
+import org.apache.spark.RDD
+import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.util.StatCounter
+import org.apache.spark.partial.{BoundedDouble, PartialResult}
+import org.apache.spark.storage.StorageLevel
 import java.lang.Double
-import spark.Partitioner
+import org.apache.spark.Partitioner
 
 class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] {
 
@@ -119,7 +119,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
   def sum(): Double = srdd.sum()
 
   /**
-   * Return a [[spark.util.StatCounter]] object that captures the mean, variance and count
+   * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and count
    * of the RDD's elements in one operation.
    */
   def stats(): StatCounter = srdd.stats()
diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
similarity index 97%
rename from core/src/main/scala/spark/api/java/JavaPairRDD.scala
rename to core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
index effe6e5e0d6cd6d9626192f07e2d811e055412f9..09da35aee66a7945a356596929161341f781cad7 100644
--- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java
+package org.apache.spark.api.java
 
 import java.util.{List => JList}
 import java.util.Comparator
@@ -30,17 +30,17 @@ import org.apache.hadoop.mapred.OutputFormat
 import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
 import org.apache.hadoop.conf.Configuration
 
-import spark.HashPartitioner
-import spark.Partitioner
-import spark.Partitioner._
-import spark.RDD
-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.rdd.OrderedRDDFunctions
-import spark.storage.StorageLevel
+import org.apache.spark.HashPartitioner
+import org.apache.spark.Partitioner
+import org.apache.spark.Partitioner._
+import org.apache.spark.RDD
+import org.apache.spark.SparkContext.rddToPairRDDFunctions
+import org.apache.spark.api.java.function.{Function2 => JFunction2}
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.partial.BoundedDouble
+import org.apache.spark.partial.PartialResult
+import org.apache.spark.rdd.OrderedRDDFunctions
+import org.apache.spark.storage.StorageLevel
 
 
 class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManifest[K],
diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
similarity index 95%
rename from core/src/main/scala/spark/api/java/JavaRDD.scala
rename to core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
index c0bf2cf56867c717a04cacdf328193f6d336f098..68cfcf59994a4e1fe9cdc2c5ca06d5efb14a7bf6 100644
--- a/core/src/main/scala/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.api.java
+package org.apache.spark.api.java
 
-import spark._
-import spark.api.java.function.{Function => JFunction}
-import spark.storage.StorageLevel
+import org.apache.spark._
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.storage.StorageLevel
 
 class JavaRDD[T](val rdd: RDD[T])(implicit val classManifest: ClassManifest[T]) extends
 JavaRDDLike[T, JavaRDD[T]] {
diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
similarity index 97%
rename from core/src/main/scala/spark/api/java/JavaRDDLike.scala
rename to core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index 2c2b138f16c2073dde18295cbe0d4473aaf0300d..1ad8514980f912095dfdcf479c1421a97fc6a169 100644
--- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
@@ -15,18 +15,18 @@
  * limitations under the License.
  */
 
-package spark.api.java
+package org.apache.spark.api.java
 
 import java.util.{List => JList, Comparator}
 import scala.Tuple2
 import scala.collection.JavaConversions._
 
 import org.apache.hadoop.io.compress.CompressionCodec
-import spark.{SparkContext, Partition, RDD, TaskContext}
-import spark.api.java.JavaPairRDD._
-import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
-import spark.partial.{PartialResult, BoundedDouble}
-import spark.storage.StorageLevel
+import org.apache.spark.{SparkContext, Partition, RDD, TaskContext}
+import org.apache.spark.api.java.JavaPairRDD._
+import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
+import org.apache.spark.partial.{PartialResult, BoundedDouble}
+import org.apache.spark.storage.StorageLevel
 import com.google.common.base.Optional
 
 
@@ -40,7 +40,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
   /** Set of partitions in this RDD. */
   def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq)
 
-  /** The [[spark.SparkContext]] that this RDD was created on. */
+  /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */
   def context: SparkContext = rdd.context
 
   /** A unique ID for this RDD (within its SparkContext). */
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
similarity index 94%
rename from core/src/main/scala/spark/api/java/JavaSparkContext.scala
rename to core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 29d57004b5643f17131a7003a97b8d8bc331acd8..618a7b3bf7f7445ed8feb52ac7250d21ed672bec 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java
+package org.apache.spark.api.java
 
 import java.util.{Map => JMap}
 
@@ -27,15 +27,15 @@ import org.apache.hadoop.mapred.InputFormat
 import org.apache.hadoop.mapred.JobConf
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
 
-import spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, RDD, SparkContext}
-import spark.SparkContext.IntAccumulatorParam
-import spark.SparkContext.DoubleAccumulatorParam
-import spark.broadcast.Broadcast
+import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, RDD, SparkContext}
+import org.apache.spark.SparkContext.IntAccumulatorParam
+import org.apache.spark.SparkContext.DoubleAccumulatorParam
+import org.apache.spark.broadcast.Broadcast
 
 import com.google.common.base.Optional
 
 /**
- * A Java-friendly version of [[spark.SparkContext]] that returns [[spark.api.java.JavaRDD]]s and
+ * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and
  * works with Java collections instead of Scala ones.
  */
 class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround {
@@ -283,48 +283,48 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
   }
 
   /**
-   * Create an [[spark.Accumulator]] integer variable, which tasks can "add" values
+   * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values
    * to using the `add` method. Only the master can access the accumulator's `value`.
    */
   def intAccumulator(initialValue: Int): Accumulator[java.lang.Integer] =
     sc.accumulator(initialValue)(IntAccumulatorParam).asInstanceOf[Accumulator[java.lang.Integer]]
 
   /**
-   * Create an [[spark.Accumulator]] double variable, which tasks can "add" values
+   * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values
    * to using the `add` method. Only the master can access the accumulator's `value`.
    */
   def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] =
     sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]]
 
   /**
-   * Create an [[spark.Accumulator]] integer variable, which tasks can "add" values
+   * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values
    * to using the `add` method. Only the master can access the accumulator's `value`.
    */
   def accumulator(initialValue: Int): Accumulator[java.lang.Integer] = intAccumulator(initialValue)
 
   /**
-   * Create an [[spark.Accumulator]] double variable, which tasks can "add" values
+   * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values
    * to using the `add` method. Only the master can access the accumulator's `value`.
    */
   def accumulator(initialValue: Double): Accumulator[java.lang.Double] =
     doubleAccumulator(initialValue)
 
   /**
-   * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values
+   * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values
    * to using the `add` method. Only the master can access the accumulator's `value`.
    */
   def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] =
     sc.accumulator(initialValue)(accumulatorParam)
 
   /**
-   * Create an [[spark.Accumulable]] shared variable of the given type, to which tasks can
+   * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks can
    * "add" values with `add`. Only the master can access the accumuable's `value`.
    */
   def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] =
     sc.accumulable(initialValue)(param)
 
   /**
-   * Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for
+   * Broadcast a read-only variable to the cluster, returning a [[org.apache.spark.Broadcast]] object for
    * reading it in distributed functions. The variable will be sent to each cluster only once.
    */
   def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value)
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java
similarity index 98%
rename from core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java
rename to core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java
index 42b1de01b1032114e345a014e38ee65b8244e4b0..c9cbce5624afc3813bbe1e86ef9c8d374d577cc8 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java;
+package org.apache.spark.api.java;
 
 import java.util.Arrays;
 import java.util.ArrayList;
diff --git a/core/src/main/scala/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala
similarity index 96%
rename from core/src/main/scala/spark/api/java/JavaUtils.scala
rename to core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala
index ffc131ac83c1ef27ff12b19707ea4eeb8bd742e8..ecbf18849ad4848f2f93f88c8e6e83d7d6d9d477 100644
--- a/core/src/main/scala/spark/api/java/JavaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java
+package org.apache.spark.api.java
 
 import com.google.common.base.Optional
 
diff --git a/core/src/main/scala/spark/api/java/StorageLevels.java b/core/src/main/scala/org/apache/spark/api/java/StorageLevels.java
similarity index 96%
rename from core/src/main/scala/spark/api/java/StorageLevels.java
rename to core/src/main/scala/org/apache/spark/api/java/StorageLevels.java
index f385636e832f75262d51413761329ce33915e118..0744269773f8d7e8bc47d55bac2bc26b44795079 100644
--- a/core/src/main/scala/spark/api/java/StorageLevels.java
+++ b/core/src/main/scala/org/apache/spark/api/java/StorageLevels.java
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.api.java;
+package org.apache.spark.api.java;
 
-import spark.storage.StorageLevel;
+import org.apache.spark.storage.StorageLevel;
 
 /**
  * Expose some commonly useful storage level constants.
diff --git a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
similarity index 96%
rename from core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java
rename to core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
index 8bc88d757fd68b4f42da6fd377ee83fb483e988f..4830067f7a0db2a7b483c5c962483a876b0788ff 100644
--- a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function;
+package org.apache.spark.api.java.function;
 
 
 import scala.runtime.AbstractFunction1;
diff --git a/core/src/main/scala/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java
similarity index 96%
rename from core/src/main/scala/spark/api/java/function/DoubleFunction.java
rename to core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java
index 1aa1e5dae03439be00c8439d4837d8ebd621b6c7..db34cd190ad3a583c73b0a1825f4118970855d03 100644
--- a/core/src/main/scala/spark/api/java/function/DoubleFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function;
+package org.apache.spark.api.java.function;
 
 
 import scala.runtime.AbstractFunction1;
diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
similarity index 96%
rename from core/src/main/scala/spark/api/java/function/FlatMapFunction.scala
rename to core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
index 9eb0cfe3f9a275711f094c3e339d2a4ad8acebe5..158539a8461f94aa85ea6440786d50d461436825 100644
--- a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function
+package org.apache.spark.api.java.function
 
 /**
  * A function that returns zero or more output records from each input record.
diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
similarity index 96%
rename from core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala
rename to core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
index dda98710c2d065d3d94e54ce2662f1f6d2ec9270..5ef6a814f5a4f2aa347dc1f7487b7011308bcfce 100644
--- a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function
+package org.apache.spark.api.java.function
 
 /**
  * A function that takes two inputs and returns zero or more output records.
diff --git a/core/src/main/scala/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java
similarity index 97%
rename from core/src/main/scala/spark/api/java/function/Function.java
rename to core/src/main/scala/org/apache/spark/api/java/function/Function.java
index 2a2ea0aacf03ccf15d9b01c101a9707a67d7c1ee..b9070cfd837eb68743042ae79500ced87815eb4c 100644
--- a/core/src/main/scala/spark/api/java/function/Function.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function;
+package org.apache.spark.api.java.function;
 
 import scala.reflect.ClassManifest;
 import scala.reflect.ClassManifest$;
diff --git a/core/src/main/scala/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
similarity index 96%
rename from core/src/main/scala/spark/api/java/function/Function2.java
rename to core/src/main/scala/org/apache/spark/api/java/function/Function2.java
index 952d31ece49167d2521883a6dc6d4077c7623b59..d4c9154869fe9ac14a4432951ca8cb4a490b5eb2 100644
--- a/core/src/main/scala/spark/api/java/function/Function2.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function;
+package org.apache.spark.api.java.function;
 
 import scala.reflect.ClassManifest;
 import scala.reflect.ClassManifest$;
diff --git a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
similarity index 97%
rename from core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java
rename to core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
index 4aad602da3a1c2dcbccea46ed6257edbe7550f73..c0e5544b7dff0d66576243d1e422bf4e57ee15dc 100644
--- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function;
+package org.apache.spark.api.java.function;
 
 import scala.Tuple2;
 import scala.reflect.ClassManifest;
diff --git a/core/src/main/scala/spark/api/java/function/PairFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
similarity index 97%
rename from core/src/main/scala/spark/api/java/function/PairFunction.java
rename to core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
index ccfe64ecf142f572517ec48952c12771c64073f2..40480fe8e816055494be1b8138a9bb487b357326 100644
--- a/core/src/main/scala/spark/api/java/function/PairFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function;
+package org.apache.spark.api.java.function;
 
 import scala.Tuple2;
 import scala.reflect.ClassManifest;
diff --git a/core/src/main/scala/spark/api/java/function/VoidFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala
similarity index 96%
rename from core/src/main/scala/spark/api/java/function/VoidFunction.scala
rename to core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala
index f6fc0b0f7dbc318903556ddc64b6ea25e9cacd08..ea94313a4ab59d113e69660030c78ba5ddfb45e6 100644
--- a/core/src/main/scala/spark/api/java/function/VoidFunction.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function
+package org.apache.spark.api.java.function
 
 /**
  * A function with no return value.
diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala
similarity index 96%
rename from core/src/main/scala/spark/api/java/function/WrappedFunction1.scala
rename to core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala
index 1758a38c4e21cb5147048a28d7c3cdb92408dd82..cfe694f65d558a16ea5cb16732904c260ef95de7 100644
--- a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function
+package org.apache.spark.api.java.function
 
 import scala.runtime.AbstractFunction1
 
diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction2.scala
similarity index 96%
rename from core/src/main/scala/spark/api/java/function/WrappedFunction2.scala
rename to core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction2.scala
index b093567d2c1bdf19e8e925ab97fa90cc58eb084c..eb9277c6fb4cba62577e8751b89387960c6236b2 100644
--- a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction2.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.java.function
+package org.apache.spark.api.java.function
 
 import scala.runtime.AbstractFunction2
 
diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala
similarity index 90%
rename from core/src/main/scala/spark/api/python/PythonPartitioner.scala
rename to core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala
index ac112b8c2c773078983d4acab3c894e53a5b9021..eea63d5a4e452d0023778e2bea6af477f8813ead 100644
--- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.api.python
+package org.apache.spark.api.python
 
-import spark.Partitioner
-import spark.Utils
+import org.apache.spark.Partitioner
+import org.apache.spark.Utils
 import java.util.Arrays
 
 /**
- * A [[spark.Partitioner]] that performs handling of byte arrays, for use by the Python API.
+ * A [[org.apache.spark.Partitioner]] that performs handling of byte arrays, for use by the Python API.
  *
  * Stores the unique id() of the Python-side partitioning function so that it is incorporated into
  * equality comparisons.  Correctness requires that the id is a unique identifier for the
diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
similarity index 97%
rename from core/src/main/scala/spark/api/python/PythonRDD.scala
rename to core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 49671437d041b329c910de6ff1cc5360c45f66a3..621f0fe8ee646faff5432598b64f5b136642d945 100644
--- a/core/src/main/scala/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.api.python
+package org.apache.spark.api.python
 
 import java.io._
 import java.net._
@@ -23,10 +23,10 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio
 
 import scala.collection.JavaConversions._
 
-import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
-import spark.broadcast.Broadcast
-import spark._
-import spark.rdd.PipedRDD
+import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark._
+import org.apache.spark.rdd.PipedRDD
 
 
 private[spark] class PythonRDD[T: ClassManifest](
@@ -298,7 +298,7 @@ private object Pickle {
   val APPENDS: Byte = 'e'
 }
 
-private class BytesToString extends spark.api.java.function.Function[Array[Byte], String] {
+private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] {
   override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8")
 }
 
diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
similarity index 98%
rename from core/src/main/scala/spark/api/python/PythonWorkerFactory.scala
rename to core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
index 14f8320678217e3e79ed1c42fc0109bd7ea4422f..08e3f670f57f6ea63feeb59b2b2a8a397acda394 100644
--- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.api.python
+package org.apache.spark.api.python
 
 import java.io.{File, DataInputStream, IOException}
 import java.net.{Socket, SocketException, InetAddress}
 
 import scala.collection.JavaConversions._
 
-import spark._
+import org.apache.spark._
 
 private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String])
     extends Logging {
diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala
similarity index 99%
rename from core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
rename to core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala
index 6f7d385379880ae0a5b45b213f1b08ee8dd4164a..99e86237fc08061a13ec0d08baa3349b421a8aed 100644
--- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.broadcast
+package org.apache.spark.broadcast
 
 import java.io._
 import java.net._
@@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicInteger
 import scala.collection.mutable.{ListBuffer, Map, Set}
 import scala.math
 
-import spark._
-import spark.storage.StorageLevel
+import org.apache.spark._
+import org.apache.spark.storage.StorageLevel
 
 private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
   extends Broadcast[T](id)
diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
similarity index 91%
rename from core/src/main/scala/spark/broadcast/Broadcast.scala
rename to core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
index aba56a60cade240cb8cc749168054383d7023f09..43c18294c552b69a77e404ca3511634b60a94612 100644
--- a/core/src/main/scala/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.broadcast
+package org.apache.spark.broadcast
 
 import java.io._
 import java.util.concurrent.atomic.AtomicLong
 
-import spark._
+import org.apache.spark._
 
 abstract class Broadcast[T](private[spark] val id: Long) extends Serializable {
   def value: T
@@ -28,7 +28,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable {
   // We cannot have an abstract readObject here due to some weird issues with
   // readObject having to be 'private' in sub-classes.
 
-  override def toString = "spark.Broadcast(" + id + ")"
+  override def toString = "Broadcast(" + id + ")"
 }
 
 private[spark] 
@@ -44,7 +44,7 @@ class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable
     synchronized {
       if (!initialized) {
         val broadcastFactoryClass = System.getProperty(
-          "spark.broadcast.factory", "spark.broadcast.HttpBroadcastFactory")
+          "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory")
 
         broadcastFactory =
           Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
similarity index 97%
rename from core/src/main/scala/spark/broadcast/BroadcastFactory.scala
rename to core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
index d33d95c7d95e8031bbe147626265bd7bf66653e7..68bff75b908c73fe11dea8a46d5ccff51258a6a3 100644
--- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.broadcast
+package org.apache.spark.broadcast
 
 /**
  * An interface for all the broadcast implementations in Spark (to allow 
diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
similarity index 95%
rename from core/src/main/scala/spark/broadcast/HttpBroadcast.scala
rename to core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
index 138a8c21bcfeb09bed73fc9cd77a435218f527f6..7a52ff0769c3755b98b9308cbdcb84482e72ef37 100644
--- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.broadcast
+package org.apache.spark.broadcast
 
 import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream}
 import java.net.URL
@@ -23,10 +23,10 @@ import java.net.URL
 import it.unimi.dsi.fastutil.io.FastBufferedInputStream
 import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
 
-import spark.{HttpServer, Logging, SparkEnv, Utils}
-import spark.io.CompressionCodec
-import spark.storage.StorageLevel
-import spark.util.{MetadataCleaner, TimeStampedHashSet}
+import org.apache.spark.{HttpServer, Logging, SparkEnv, Utils}
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.{MetadataCleaner, TimeStampedHashSet}
 
 
 private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala
similarity index 99%
rename from core/src/main/scala/spark/broadcast/MultiTracker.scala
rename to core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala
index 7855d44e9b0a603f49944896ed0ef5386a375568..10b910df87e506f8f9303bba947c137d41517ea3 100644
--- a/core/src/main/scala/spark/broadcast/MultiTracker.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.broadcast
+package org.apache.spark.broadcast
 
 import java.io._
 import java.net._
@@ -23,7 +23,7 @@ import java.util.Random
 
 import scala.collection.mutable.Map
 
-import spark._
+import org.apache.spark._
 
 private object MultiTracker
 extends Logging {
diff --git a/core/src/main/scala/spark/broadcast/SourceInfo.scala b/core/src/main/scala/org/apache/spark/broadcast/SourceInfo.scala
similarity index 96%
rename from core/src/main/scala/spark/broadcast/SourceInfo.scala
rename to core/src/main/scala/org/apache/spark/broadcast/SourceInfo.scala
index b17ae63b5c43b8ad83bfb093a2cc02967cb5a133..baa1fd6da46e820ed1d706198d4ab742bb4c19b6 100644
--- a/core/src/main/scala/spark/broadcast/SourceInfo.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/SourceInfo.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.broadcast
+package org.apache.spark.broadcast
 
 import java.util.BitSet
 
-import spark._
+import org.apache.spark._
 
 /**
  * Used to keep and pass around information of peers involved in a broadcast
diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala
similarity index 99%
rename from core/src/main/scala/spark/broadcast/TreeBroadcast.scala
rename to core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala
index ea1e9a12c1bfdaacf408986e8d744e761437368b..b5a4ccc0eee1b37572ca7d6e56bae08fe7ebd7b3 100644
--- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.broadcast
+package org.apache.spark.broadcast
 
 import java.io._
 import java.net._
@@ -24,8 +24,8 @@ import java.util.{Comparator, Random, UUID}
 import scala.collection.mutable.{ListBuffer, Map, Set}
 import scala.math
 
-import spark._
-import spark.storage.StorageLevel
+import org.apache.spark._
+import org.apache.spark.storage.StorageLevel
 
 private[spark] class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
 extends Broadcast[T](id) with Logging with Serializable {
diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
similarity index 97%
rename from core/src/main/scala/spark/deploy/ApplicationDescription.scala
rename to core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
index a8b22fbef877771b306fcf6cde1332d7488c4514..19d393a0dbce7ac92f58a91be04334fd47fe02a4 100644
--- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy
+package org.apache.spark.deploy
 
 private[spark] class ApplicationDescription(
     val name: String,
diff --git a/core/src/main/scala/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala
similarity index 96%
rename from core/src/main/scala/spark/deploy/Command.scala
rename to core/src/main/scala/org/apache/spark/deploy/Command.scala
index bad629e96529c946482e011e8024c4a23531af67..fa8af9a6467509a35f9c58d7b98782abf358aec9 100644
--- a/core/src/main/scala/spark/deploy/Command.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy
+package org.apache.spark.deploy
 
 import scala.collection.Map
 
diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
similarity index 93%
rename from core/src/main/scala/spark/deploy/DeployMessage.scala
rename to core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 0db13ffc98723ff59183e02212a1e13a5575a5b9..4dc6ada2d1649bb19e40794d00d78b0d779d1386 100644
--- a/core/src/main/scala/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.deploy
+package org.apache.spark.deploy
 
 import scala.collection.immutable.List
 
-import spark.Utils
-import spark.deploy.ExecutorState.ExecutorState
-import spark.deploy.master.{WorkerInfo, ApplicationInfo}
-import spark.deploy.worker.ExecutorRunner
+import org.apache.spark.Utils
+import org.apache.spark.deploy.ExecutorState.ExecutorState
+import org.apache.spark.deploy.master.{WorkerInfo, ApplicationInfo}
+import org.apache.spark.deploy.worker.ExecutorRunner
 
 
 private[deploy] sealed trait DeployMessage extends Serializable
diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala
similarity index 97%
rename from core/src/main/scala/spark/deploy/ExecutorState.scala
rename to core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala
index 08c9a3b7252b53a087d84f987d99aeac9c217027..fcfea96ad60b88f5863d0323d9019f91052082ee 100644
--- a/core/src/main/scala/spark/deploy/ExecutorState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy
+package org.apache.spark.deploy
 
 private[spark] object ExecutorState
   extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") {
diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
similarity index 92%
rename from core/src/main/scala/spark/deploy/JsonProtocol.scala
rename to core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
index f8dcf025b44ecc4e0c65bb367199939cce5cbc4e..a6be8efef18f6e44ae681b58864ec5dd121d0e69 100644
--- a/core/src/main/scala/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.deploy
+package org.apache.spark.deploy
 
 import net.liftweb.json.JsonDSL._
 
-import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
-import spark.deploy.master.{ApplicationInfo, WorkerInfo}
-import spark.deploy.worker.ExecutorRunner
+import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
+import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo}
+import org.apache.spark.deploy.worker.ExecutorRunner
 
 
 private[spark] object JsonProtocol {
diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
similarity index 92%
rename from core/src/main/scala/spark/deploy/LocalSparkCluster.scala
rename to core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index 6b8e9f27af7c8d779a5467ac66f54631d3675030..af5a4110b07135728a431796b55035f7aecfe93e 100644
--- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.deploy
+package org.apache.spark.deploy
 
 import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
 
-import spark.deploy.worker.Worker
-import spark.deploy.master.Master
-import spark.util.AkkaUtils
-import spark.{Logging, Utils}
+import org.apache.spark.deploy.worker.Worker
+import org.apache.spark.deploy.master.Master
+import org.apache.spark.util.AkkaUtils
+import org.apache.spark.{Logging, Utils}
 
 import scala.collection.mutable.ArrayBuffer
 
diff --git a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
similarity index 97%
rename from core/src/main/scala/spark/deploy/SparkHadoopUtil.scala
rename to core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 882161e66997005b61ea3120ffee0023f58bd010..0a5f4c368f3d1ef12374f5842f4bd7b0d0354817 100644
--- a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy
+package org.apache.spark.deploy
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapred.JobConf
 
diff --git a/core/src/main/scala/spark/deploy/WebUI.scala b/core/src/main/scala/org/apache/spark/deploy/WebUI.scala
similarity index 98%
rename from core/src/main/scala/spark/deploy/WebUI.scala
rename to core/src/main/scala/org/apache/spark/deploy/WebUI.scala
index 8ea7792ef4ecef106796d39a15881ce8c822cfb2..ae258b58b9cc5554ecc883ade04f9aa87fc98c5e 100644
--- a/core/src/main/scala/spark/deploy/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/WebUI.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy
+package org.apache.spark.deploy
 
 import java.text.SimpleDateFormat
 import java.util.Date
diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
similarity index 95%
rename from core/src/main/scala/spark/deploy/client/Client.scala
rename to core/src/main/scala/org/apache/spark/deploy/client/Client.scala
index 9d5ba8a796013a0ffb5ceeab0eb3fe1461811560..a342dd724a8ed74bbc1b80bc5d2eb309ded1e7c6 100644
--- a/core/src/main/scala/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.client
+package org.apache.spark.deploy.client
 
 import java.util.concurrent.TimeoutException
 
@@ -28,10 +28,10 @@ import akka.remote.RemoteClientLifeCycleEvent
 import akka.remote.RemoteClientShutdown
 import akka.dispatch.Await
 
-import spark.Logging
-import spark.deploy.{ApplicationDescription, ExecutorState}
-import spark.deploy.DeployMessages._
-import spark.deploy.master.Master
+import org.apache.spark.Logging
+import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
+import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.master.Master
 
 
 /**
diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
similarity index 97%
rename from core/src/main/scala/spark/deploy/client/ClientListener.scala
rename to core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
index 064024455ee068ce8a43c52777b5f12c56d280bc..4605368c1177f56639356639f3aae9951aaee06e 100644
--- a/core/src/main/scala/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.client
+package org.apache.spark.deploy.client
 
 /**
  * Callbacks invoked by deploy client when various events happen. There are currently four events:
diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
similarity index 90%
rename from core/src/main/scala/spark/deploy/client/TestClient.scala
rename to core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index 4f4daa141a4f248652ff51fe949c32b0938bcfd8..0322029fbd00336972d048856cef47e3389c6876 100644
--- a/core/src/main/scala/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.deploy.client
+package org.apache.spark.deploy.client
 
-import spark.util.AkkaUtils
-import spark.{Logging, Utils}
-import spark.deploy.{Command, ApplicationDescription}
+import org.apache.spark.util.AkkaUtils
+import org.apache.spark.{Logging, Utils}
+import org.apache.spark.deploy.{Command, ApplicationDescription}
 
 private[spark] object TestClient {
 
diff --git a/core/src/main/scala/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala
similarity index 96%
rename from core/src/main/scala/spark/deploy/client/TestExecutor.scala
rename to core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala
index 8a22b6b89fa8e6af278ac5bd8ae78f844ef8833f..c5ac45c6730d3f45f41b200d88ee62f96d16c0c1 100644
--- a/core/src/main/scala/spark/deploy/client/TestExecutor.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.client
+package org.apache.spark.deploy.client
 
 private[spark] object TestExecutor {
   def main(args: Array[String]) {
diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
similarity index 96%
rename from core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
index 6dd2f06126527dd5c3dae462b07f9978fc23f502..bd5327627a832fc3b28202920f889bbd421d7ec5 100644
--- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.deploy.master
+package org.apache.spark.deploy.master
 
-import spark.deploy.ApplicationDescription
+import org.apache.spark.deploy.ApplicationDescription
 import java.util.Date
 import akka.actor.ActorRef
 import scala.collection.mutable
diff --git a/core/src/main/scala/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala
similarity index 89%
rename from core/src/main/scala/spark/deploy/master/ApplicationSource.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala
index 4df2b6bfddd7503ef4b5b59b8e9b3c660ddfd3ec..2d75ad5a2c9af56689a726d9632d6ebbff12bcce 100644
--- a/core/src/main/scala/spark/deploy/master/ApplicationSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala
@@ -1,8 +1,8 @@
-package spark.deploy.master
+package org.apache.spark.deploy.master
 
 import com.codahale.metrics.{Gauge, MetricRegistry}
 
-import spark.metrics.source.Source
+import org.apache.spark.metrics.source.Source
 
 class ApplicationSource(val application: ApplicationInfo) extends Source {
   val metricRegistry = new MetricRegistry()
diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
similarity index 96%
rename from core/src/main/scala/spark/deploy/master/ApplicationState.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
index 94f0ad8baec476000a17c31360530872ca9035a3..7e804223cf48a6459a7c60caa50cff33e7675d89 100644
--- a/core/src/main/scala/spark/deploy/master/ApplicationState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.master
+package org.apache.spark.deploy.master
 
 private[spark] object ApplicationState
   extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") {
diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
similarity index 92%
rename from core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
index 99b60f7d092a42c72ea0649e9667d111c34394fd..cf384a985e90ede61afa0eb7267d636af0eada5e 100644
--- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.deploy.master
+package org.apache.spark.deploy.master
 
-import spark.deploy.ExecutorState
+import org.apache.spark.deploy.ExecutorState
 
 private[spark] class ExecutorInfo(
     val id: Int,
diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
similarity index 97%
rename from core/src/main/scala/spark/deploy/master/Master.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 04af5e149c62d3431c27095ac66ff3152c431e91..869b2b2646d04c78035f38e96eafd338326d3867 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.master
+package org.apache.spark.deploy.master
 
 import java.text.SimpleDateFormat
 import java.util.Date
@@ -27,12 +27,12 @@ import akka.actor.Terminated
 import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
 import akka.util.duration._
 
-import spark.{Logging, SparkException, Utils}
-import spark.deploy.{ApplicationDescription, ExecutorState}
-import spark.deploy.DeployMessages._
-import spark.deploy.master.ui.MasterWebUI
-import spark.metrics.MetricsSystem
-import spark.util.AkkaUtils
+import org.apache.spark.{Logging, SparkException, Utils}
+import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
+import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.master.ui.MasterWebUI
+import org.apache.spark.metrics.MetricsSystem
+import org.apache.spark.util.AkkaUtils
 
 
 private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
similarity index 96%
rename from core/src/main/scala/spark/deploy/master/MasterArguments.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
index 0ae01607673797263aa2844771d3b42d86a2a1ae..c86cca278d04d665a0aed1d1027ab022c22a5157 100644
--- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.deploy.master
+package org.apache.spark.deploy.master
 
-import spark.util.IntParam
-import spark.Utils
+import org.apache.spark.util.IntParam
+import org.apache.spark.Utils
 
 /**
  * Command-line parser for the master.
diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
similarity index 90%
rename from core/src/main/scala/spark/deploy/master/MasterSource.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
index b8cfa6a7736841cd582ae5a8f2aa2fe71506f578..8dd0a42f717c7a2ddc278baa1e9c60ba3101f21a 100644
--- a/core/src/main/scala/spark/deploy/master/MasterSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
@@ -1,8 +1,8 @@
-package spark.deploy.master
+package org.apache.spark.deploy.master
 
 import com.codahale.metrics.{Gauge, MetricRegistry}
 
-import spark.metrics.source.Source
+import org.apache.spark.metrics.source.Source
 
 private[spark] class MasterSource(val master: Master) extends Source {
   val metricRegistry = new MetricRegistry()
diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
similarity index 96%
rename from core/src/main/scala/spark/deploy/master/WorkerInfo.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index 4135cfeb2843f8d9a385b9830422cac428f0ce35..285e07a823213f2dc33b17a136de60397c0df0ed 100644
--- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.deploy.master
+package org.apache.spark.deploy.master
 
 import akka.actor.ActorRef
 import scala.collection.mutable
-import spark.Utils
+import org.apache.spark.Utils
 
 private[spark] class WorkerInfo(
   val id: String,
diff --git a/core/src/main/scala/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
similarity index 96%
rename from core/src/main/scala/spark/deploy/master/WorkerState.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
index 3e50b7748d6683c69fb714cb45f611397431c139..b5ee6dca79fab36aeace009d436d7fd1ea69e481 100644
--- a/core/src/main/scala/spark/deploy/master/WorkerState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.master
+package org.apache.spark.deploy.master
 
 private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") {
   type WorkerState = Value
diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
similarity index 93%
rename from core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index 2ad98f759cb8a9afca50ce5130f5430765b33cd3..6435c7f917c87417f557b86cf09b1d46d58c1c88 100644
--- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.master.ui
+package org.apache.spark.deploy.master.ui
 
 import scala.xml.Node
 
@@ -27,11 +27,11 @@ import javax.servlet.http.HttpServletRequest
 
 import net.liftweb.json.JsonAST.JValue
 
-import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
-import spark.deploy.JsonProtocol
-import spark.deploy.master.ExecutorInfo
-import spark.ui.UIUtils
-import spark.Utils
+import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
+import org.apache.spark.deploy.JsonProtocol
+import org.apache.spark.deploy.master.ExecutorInfo
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.Utils
 
 private[spark] class ApplicationPage(parent: MasterWebUI) {
   val master = parent.masterActorRef
diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
similarity index 93%
rename from core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index 093e523e2392eb52e16900f551b240e68c221b02..58d3863009eb0c9923f4b2e64f0c7d39d8672fbc 100644
--- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.master.ui
+package org.apache.spark.deploy.master.ui
 
 import javax.servlet.http.HttpServletRequest
 
@@ -27,12 +27,12 @@ import akka.util.duration._
 
 import net.liftweb.json.JsonAST.JValue
 
-import spark.Utils
-import spark.deploy.DeployWebUI
-import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
-import spark.deploy.JsonProtocol
-import spark.deploy.master.{ApplicationInfo, WorkerInfo}
-import spark.ui.UIUtils
+import org.apache.spark.Utils
+import org.apache.spark.deploy.DeployWebUI
+import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
+import org.apache.spark.deploy.JsonProtocol
+import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo}
+import org.apache.spark.ui.UIUtils
 
 private[spark] class IndexPage(parent: MasterWebUI) {
   val master = parent.masterActorRef
diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
similarity index 90%
rename from core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala
rename to core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index c91e1db9f24e67ecc59dbea91a7458d8d3a8c4a0..47b1e521f5c84f9283b076fef7896614ea12b286 100644
--- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.master.ui
+package org.apache.spark.deploy.master.ui
 
 import akka.util.Duration
 
@@ -23,10 +23,10 @@ import javax.servlet.http.HttpServletRequest
 
 import org.eclipse.jetty.server.{Handler, Server}
 
-import spark.{Logging, Utils}
-import spark.deploy.master.Master
-import spark.ui.JettyUtils
-import spark.ui.JettyUtils._
+import org.apache.spark.{Logging, Utils}
+import org.apache.spark.deploy.master.Master
+import org.apache.spark.ui.JettyUtils
+import org.apache.spark.ui.JettyUtils._
 
 /**
  * Web UI server for the standalone master.
@@ -76,5 +76,5 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging {
 }
 
 private[spark] object MasterWebUI {
-  val STATIC_RESOURCE_DIR = "spark/ui/static"
+  val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static"
 }
diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
similarity index 96%
rename from core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
rename to core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index 34665ce451a4a1c1e891e08b7493456a44c3272e..01ce4a6deab5940d6a78266f84c68ac875ec48d6 100644
--- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.worker
+package org.apache.spark.deploy.worker
 
 import java.io._
 import java.lang.System.getenv
@@ -25,9 +25,9 @@ import akka.actor.ActorRef
 import com.google.common.base.Charsets
 import com.google.common.io.Files
 
-import spark.{Utils, Logging}
-import spark.deploy.{ExecutorState, ApplicationDescription}
-import spark.deploy.DeployMessages.ExecutorStateChanged
+import org.apache.spark.{Utils, Logging}
+import org.apache.spark.deploy.{ExecutorState, ApplicationDescription}
+import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged
 
 /**
  * Manages the execution of one executor process.
diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
similarity index 95%
rename from core/src/main/scala/spark/deploy/worker/Worker.scala
rename to core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 053ac5522607b0d9bd57df2d4501163adae0806c..86e8e7543ba1215653d0a8c7e4d960e9be1a256d 100644
--- a/core/src/main/scala/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.worker
+package org.apache.spark.deploy.worker
 
 import java.text.SimpleDateFormat
 import java.util.Date
@@ -27,13 +27,13 @@ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
 import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
 import akka.util.duration._
 
-import spark.{Logging, Utils}
-import spark.deploy.ExecutorState
-import spark.deploy.DeployMessages._
-import spark.deploy.master.Master
-import spark.deploy.worker.ui.WorkerWebUI
-import spark.metrics.MetricsSystem
-import spark.util.AkkaUtils
+import org.apache.spark.{Logging, Utils}
+import org.apache.spark.deploy.ExecutorState
+import org.apache.spark.deploy.DeployMessages._
+import org.apache.spark.deploy.master.Master
+import org.apache.spark.deploy.worker.ui.WorkerWebUI
+import org.apache.spark.metrics.MetricsSystem
+import org.apache.spark.util.AkkaUtils
 
 
 private[spark] class Worker(
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
similarity index 97%
rename from core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
rename to core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
index 9fcd3260ca97529019d41a9049700e093bd59abd..6d91223413e73ba043e74fc6ecf2a8a8343c611b 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.deploy.worker
+package org.apache.spark.deploy.worker
 
-import spark.util.IntParam
-import spark.util.MemoryParam
-import spark.Utils
+import org.apache.spark.util.IntParam
+import org.apache.spark.util.MemoryParam
+import org.apache.spark.Utils
 import java.lang.management.ManagementFactory
 
 /**
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala
similarity index 92%
rename from core/src/main/scala/spark/deploy/worker/WorkerSource.scala
rename to core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala
index 39cb8e56901a1f36178f2e1e3ba4d6fb490d74e0..6427c0178fa008f2db4d454edbcb7f012d0cb5f1 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala
@@ -1,8 +1,8 @@
-package spark.deploy.worker
+package org.apache.spark.deploy.worker
 
 import com.codahale.metrics.{Gauge, MetricRegistry}
 
-import spark.metrics.source.Source
+import org.apache.spark.metrics.source.Source
 
 private[spark] class WorkerSource(val worker: Worker) extends Source {
   val sourceName = "worker"
diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
similarity index 93%
rename from core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala
rename to core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index 243e0765cb7d0619750f6ec94ce84eddef96e157..6192c2324bd714857cee01de5deb1824b1a233d4 100644
--- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.worker.ui
+package org.apache.spark.deploy.worker.ui
 
 import javax.servlet.http.HttpServletRequest
 
@@ -27,11 +27,11 @@ import akka.util.duration._
 
 import net.liftweb.json.JsonAST.JValue
 
-import spark.Utils
-import spark.deploy.JsonProtocol
-import spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse}
-import spark.deploy.worker.ExecutorRunner
-import spark.ui.UIUtils
+import org.apache.spark.Utils
+import org.apache.spark.deploy.JsonProtocol
+import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse}
+import org.apache.spark.deploy.worker.ExecutorRunner
+import org.apache.spark.ui.UIUtils
 
 
 private[spark] class IndexPage(parent: WorkerWebUI) {
diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
similarity index 95%
rename from core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala
rename to core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index 0a75ad8cf4a0ab05051dc99a356631b17c4e8bff..bb8165ac0988626117e0266345464cadf0f7bc73 100644
--- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.worker.ui
+package org.apache.spark.deploy.worker.ui
 
 import akka.util.{Duration, Timeout}
 
@@ -25,11 +25,11 @@ import javax.servlet.http.HttpServletRequest
 
 import org.eclipse.jetty.server.{Handler, Server}
 
-import spark.deploy.worker.Worker
-import spark.{Utils, Logging}
-import spark.ui.JettyUtils
-import spark.ui.JettyUtils._
-import spark.ui.UIUtils
+import org.apache.spark.deploy.worker.Worker
+import org.apache.spark.{Utils, Logging}
+import org.apache.spark.ui.JettyUtils
+import org.apache.spark.ui.JettyUtils._
+import org.apache.spark.ui.UIUtils
 
 /**
  * Web UI server for the standalone worker.
@@ -185,6 +185,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
 }
 
 private[spark] object WorkerWebUI {
-  val STATIC_RESOURCE_DIR = "spark/ui/static"
+  val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static"
   val DEFAULT_PORT="8081"
 }
diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
similarity index 97%
rename from core/src/main/scala/spark/executor/Executor.scala
rename to core/src/main/scala/org/apache/spark/executor/Executor.scala
index fa82d2b3245dade349329cf8120c62ae1433fc11..5446a3fca9f67a467669a3225ce15fb71198a98b 100644
--- a/core/src/main/scala/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.executor
+package org.apache.spark.executor
 
 import java.io.{File}
 import java.lang.management.ManagementFactory
@@ -25,8 +25,8 @@ import java.util.concurrent._
 import scala.collection.JavaConversions._
 import scala.collection.mutable.HashMap
 
-import spark.scheduler._
-import spark._
+import org.apache.spark.scheduler._
+import org.apache.spark._
 
 
 /**
@@ -225,13 +225,13 @@ private[spark] class Executor(
     if (classUri != null) {
       logInfo("Using REPL class URI: " + classUri)
       try {
-        val klass = Class.forName("spark.repl.ExecutorClassLoader")
+        val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader")
           .asInstanceOf[Class[_ <: ClassLoader]]
         val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader])
         return constructor.newInstance(classUri, parent)
       } catch {
         case _: ClassNotFoundException =>
-          logError("Could not find spark.repl.ExecutorClassLoader on classpath!")
+          logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!")
           System.exit(1)
           null
       }
diff --git a/core/src/main/scala/spark/executor/ExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala
similarity index 93%
rename from core/src/main/scala/spark/executor/ExecutorBackend.scala
rename to core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala
index 33a6f8a824e709a504a239ae6559841d326ca652..ad7dd34c769404bea090af0faa56ed95b9d2b323 100644
--- a/core/src/main/scala/spark/executor/ExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.executor
+package org.apache.spark.executor
 
 import java.nio.ByteBuffer
-import spark.TaskState.TaskState
+import org.apache.spark.TaskState.TaskState
 
 /**
  * A pluggable interface used by the Executor to send updates to the cluster scheduler.
diff --git a/core/src/main/scala/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala
similarity index 98%
rename from core/src/main/scala/spark/executor/ExecutorExitCode.scala
rename to core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala
index 64b9fb88f898297642a1a9ca4b7bff11ff33e04d..e5c9bbbe2874e66b8fff8a16126bd1623138d8f2 100644
--- a/core/src/main/scala/spark/executor/ExecutorExitCode.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.executor
+package org.apache.spark.executor
 
 /**
  * These are exit codes that executors should use to provide the master with information about
diff --git a/core/src/main/scala/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
similarity index 96%
rename from core/src/main/scala/spark/executor/ExecutorSource.scala
rename to core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
index d491a3c0c9599140f27f9fb1508e5453520ac77d..17653cd5608fbacc3cbb4d89ac1818fa300d90cc 100644
--- a/core/src/main/scala/spark/executor/ExecutorSource.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
@@ -1,4 +1,4 @@
-package spark.executor
+package org.apache.spark.executor
 
 import com.codahale.metrics.{Gauge, MetricRegistry}
 
@@ -8,7 +8,7 @@ import org.apache.hadoop.fs.LocalFileSystem
 
 import scala.collection.JavaConversions._
 
-import spark.metrics.source.Source
+import org.apache.spark.metrics.source.Source
 
 class ExecutorSource(val executor: Executor) extends Source {
   private def fileStats(scheme: String) : Option[FileSystem.Statistics] =
diff --git a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala
similarity index 97%
rename from core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala
rename to core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala
index 09d12fb65b9a6e94f94c8aa65a8a7ff29a1e389d..f9bfe8ed2f5baa2a60c94c3b6d765bcefa3bd557 100644
--- a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.executor
+package org.apache.spark.executor
 
 import java.net.{URLClassLoader, URL}
 
diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
similarity index 95%
rename from core/src/main/scala/spark/executor/MesosExecutorBackend.scala
rename to core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
index 4961c42faddbcd3dd69a59cb4f509e99ba43ef0b..410a94df6bf517307f8ea948b16d971b85f76604 100644
--- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.executor
+package org.apache.spark.executor
 
 import java.nio.ByteBuffer
 import org.apache.mesos.{Executor => MesosExecutor, MesosExecutorDriver, MesosNativeLibrary, ExecutorDriver}
 import org.apache.mesos.Protos.{TaskState => MesosTaskState, TaskStatus => MesosTaskStatus, _}
-import spark.TaskState.TaskState
+import org.apache.spark.TaskState.TaskState
 import com.google.protobuf.ByteString
-import spark.{Utils, Logging}
-import spark.TaskState
+import org.apache.spark.{Utils, Logging}
+import org.apache.spark.TaskState
 
 private[spark] class MesosExecutorBackend
   extends MesosExecutor
diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
similarity index 94%
rename from core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
rename to core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
index b5fb6dbe29ec0da147e7354276722bc0ac287c0e..65801f75b7d426b17ed7948523aea058ffcf4724 100644
--- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package spark.executor
+package org.apache.spark.executor
 
 import java.nio.ByteBuffer
 
 import akka.actor.{ActorRef, Actor, Props, Terminated}
 import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
 
-import spark.{Logging, Utils, SparkEnv}
-import spark.TaskState.TaskState
-import spark.scheduler.cluster.StandaloneClusterMessages._
-import spark.util.AkkaUtils
+import org.apache.spark.{Logging, Utils, SparkEnv}
+import org.apache.spark.TaskState.TaskState
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._
+import org.apache.spark.util.AkkaUtils
 
 
 private[spark] class StandaloneExecutorBackend(
diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
similarity index 98%
rename from core/src/main/scala/spark/executor/TaskMetrics.scala
rename to core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
index 47b8890bee96dc1ef0a2d1fdb7c6e1418e20960f..f311141148cb0c03fd6049c5f7df8c56cb566818 100644
--- a/core/src/main/scala/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.executor
+package org.apache.spark.executor
 
 class TaskMetrics extends Serializable {
   /**
diff --git a/core/src/main/scala/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
similarity index 94%
rename from core/src/main/scala/spark/io/CompressionCodec.scala
rename to core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 0adebecadb3e5bde51799ea6816ae77e8e51b987..90a0420cafb8f4b11400fdd643bd3b4c7f3860ed 100644
--- a/core/src/main/scala/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.io
+package org.apache.spark.io
 
 import java.io.{InputStream, OutputStream}
 
@@ -55,7 +55,7 @@ private[spark] object CompressionCodec {
 
 
 /**
- * LZF implementation of [[spark.io.CompressionCodec]].
+ * LZF implementation of [[org.apache.spark.io.CompressionCodec]].
  */
 class LZFCompressionCodec extends CompressionCodec {
 
@@ -68,7 +68,7 @@ class LZFCompressionCodec extends CompressionCodec {
 
 
 /**
- * Snappy implementation of [[spark.io.CompressionCodec]].
+ * Snappy implementation of [[org.apache.spark.io.CompressionCodec]].
  * Block size can be configured by spark.io.compression.snappy.block.size.
  */
 class SnappyCompressionCodec extends CompressionCodec {
diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
similarity index 95%
rename from core/src/main/scala/spark/metrics/MetricsConfig.scala
rename to core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
index d7fb5378a495aa1488970a035f3a9653c75321f6..0f9c4e00b1fb9ca4d77bad33e4ec317cd6e2fd6f 100644
--- a/core/src/main/scala/spark/metrics/MetricsConfig.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.metrics
+package org.apache.spark.metrics
 
 import java.util.Properties
 import java.io.{File, FileInputStream, InputStream, IOException}
@@ -23,7 +23,7 @@ import java.io.{File, FileInputStream, InputStream, IOException}
 import scala.collection.mutable
 import scala.util.matching.Regex
 
-import spark.Logging
+import org.apache.spark.Logging
 
 private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging {
   initLogging()
@@ -36,7 +36,7 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi
   var propertyCategories: mutable.HashMap[String, Properties] = null
 
   private def setDefaultProperties(prop: Properties) {
-    prop.setProperty("*.sink.servlet.class", "spark.metrics.sink.MetricsServlet")
+    prop.setProperty("*.sink.servlet.class", "org.apache.spark.metrics.sink.MetricsServlet")
     prop.setProperty("*.sink.servlet.uri", "/metrics/json")
     prop.setProperty("*.sink.servlet.sample", "false")
     prop.setProperty("master.sink.servlet.uri", "/metrics/master/json")
diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
similarity index 97%
rename from core/src/main/scala/spark/metrics/MetricsSystem.scala
rename to core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index 4e6c6b26c86be236819f3f73945dc0cbf97f5ac6..bec0c83be8bea24c4c69bc14ad07c72cbf685329 100644
--- a/core/src/main/scala/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.metrics
+package org.apache.spark.metrics
 
 import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry}
 
@@ -24,9 +24,9 @@ import java.util.concurrent.TimeUnit
 
 import scala.collection.mutable
 
-import spark.Logging
-import spark.metrics.sink.{MetricsServlet, Sink}
-import spark.metrics.source.Source
+import org.apache.spark.Logging
+import org.apache.spark.metrics.sink.{MetricsServlet, Sink}
+import org.apache.spark.metrics.source.Source
 
 /**
  * Spark Metrics System, created by specific "instance", combined by source,
diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala
similarity index 95%
rename from core/src/main/scala/spark/metrics/sink/ConsoleSink.scala
rename to core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala
index 966ba37c20e8f618e77350cb2be8b024f0b12366..bce257d6e6f47bbd4e582baedb21410aa639b412 100644
--- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.metrics.sink
+package org.apache.spark.metrics.sink
 
 import com.codahale.metrics.{ConsoleReporter, MetricRegistry}
 
 import java.util.Properties
 import java.util.concurrent.TimeUnit
 
-import spark.metrics.MetricsSystem
+import org.apache.spark.metrics.MetricsSystem
 
 class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink {
   val CONSOLE_DEFAULT_PERIOD = 10
diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala
similarity index 96%
rename from core/src/main/scala/spark/metrics/sink/CsvSink.scala
rename to core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala
index cb990afdefc4c4b3261bcd8386d19baa57221269..3d1a06a395a724fe5f6f6496c30fd4fe9e47669f 100644
--- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.metrics.sink
+package org.apache.spark.metrics.sink
 
 import com.codahale.metrics.{CsvReporter, MetricRegistry}
 
@@ -23,7 +23,7 @@ import java.io.File
 import java.util.{Locale, Properties}
 import java.util.concurrent.TimeUnit
 
-import spark.metrics.MetricsSystem
+import org.apache.spark.metrics.MetricsSystem
 
 class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink {
   val CSV_KEY_PERIOD = "period"
diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala
similarity index 96%
rename from core/src/main/scala/spark/metrics/sink/JmxSink.scala
rename to core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala
index ee04544c0e9ed8fcf9eade3347d30a937b0c93b4..621d086d415ccb920ab85dffe4023916246ba5cb 100644
--- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.metrics.sink
+package org.apache.spark.metrics.sink
 
 import com.codahale.metrics.{JmxReporter, MetricRegistry}
 
diff --git a/core/src/main/scala/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala
similarity index 96%
rename from core/src/main/scala/spark/metrics/sink/MetricsServlet.scala
rename to core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala
index 17432b1ed1a56e6c7d58351ad33e406357374067..4e90dd4323951a2d151b4c4ccd84983a940f8cb0 100644
--- a/core/src/main/scala/spark/metrics/sink/MetricsServlet.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.metrics.sink
+package org.apache.spark.metrics.sink
 
 import com.codahale.metrics.MetricRegistry
 import com.codahale.metrics.json.MetricsModule
@@ -28,7 +28,7 @@ import javax.servlet.http.HttpServletRequest
 
 import org.eclipse.jetty.server.Handler
 
-import spark.ui.JettyUtils
+import org.apache.spark.ui.JettyUtils
 
 class MetricsServlet(val property: Properties, val registry: MetricRegistry) extends Sink {
   val SERVLET_KEY_URI = "uri"
diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala
similarity index 95%
rename from core/src/main/scala/spark/metrics/sink/Sink.scala
rename to core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala
index dad1a7f0fe7022e1e021cecf020df1628c13649e..3a739aa563eaee13844be80d774854472b11712f 100644
--- a/core/src/main/scala/spark/metrics/sink/Sink.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.metrics.sink
+package org.apache.spark.metrics.sink
 
 trait Sink {
   def start: Unit
   def stop: Unit
-}
\ No newline at end of file
+}
diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala
similarity index 96%
rename from core/src/main/scala/spark/metrics/source/JvmSource.scala
rename to core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala
index e7710085577b5cd586de3306a979c1e5b65d3fff..75cb2b8973aa1590ec061219cdde158952e5615e 100644
--- a/core/src/main/scala/spark/metrics/source/JvmSource.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.metrics.source
+package org.apache.spark.metrics.source
 
 import com.codahale.metrics.MetricRegistry
 import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet}
diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala
similarity index 95%
rename from core/src/main/scala/spark/metrics/source/Source.scala
rename to core/src/main/scala/org/apache/spark/metrics/source/Source.scala
index 76199a004b560e708aa5d4cb0dd5bda79b90b51e..3fee55cc6dcd5478194d597b32f473ac02645b96 100644
--- a/core/src/main/scala/spark/metrics/source/Source.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.metrics.source
+package org.apache.spark.metrics.source
 
 import com.codahale.metrics.MetricRegistry
 
diff --git a/core/src/main/scala/spark/network/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala
similarity index 97%
rename from core/src/main/scala/spark/network/BufferMessage.scala
rename to core/src/main/scala/org/apache/spark/network/BufferMessage.scala
index e566aeac13779c76726857f73fe4ac02adbfd0f4..f736bb3713061f5258fddde2331a973dbd77ed76 100644
--- a/core/src/main/scala/spark/network/BufferMessage.scala
+++ b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
 import java.nio.ByteBuffer
 
 import scala.collection.mutable.ArrayBuffer
 
-import spark.storage.BlockManager
+import org.apache.spark.storage.BlockManager
 
 
 private[spark]
diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala
similarity index 99%
rename from core/src/main/scala/spark/network/Connection.scala
rename to core/src/main/scala/org/apache/spark/network/Connection.scala
index 1e571d39ae5763e756bc49509c1dfbaeb104030e..95cb0206acd62e67a80edcb057743d29615e4500 100644
--- a/core/src/main/scala/spark/network/Connection.scala
+++ b/core/src/main/scala/org/apache/spark/network/Connection.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
-import spark._
+import org.apache.spark._
 
 import scala.collection.mutable.{HashMap, Queue, ArrayBuffer}
 
diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
similarity index 99%
rename from core/src/main/scala/spark/network/ConnectionManager.scala
rename to core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
index 8b9f3ae18c5c90a395f33e800d2861dc29ba31be..9e2233c07b524d64ca6185d448528a7515403b31 100644
--- a/core/src/main/scala/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
-import spark._
+import org.apache.spark._
 
 import java.nio._
 import java.nio.channels._
diff --git a/core/src/main/scala/spark/network/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala
similarity index 95%
rename from core/src/main/scala/spark/network/ConnectionManagerId.scala
rename to core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala
index 9d5c518293d67b60d9275a2f47a10d5d658d7044..0839c011b8f3de32e1596a689a9bbbb1213a95ee 100644
--- a/core/src/main/scala/spark/network/ConnectionManagerId.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
 import java.net.InetSocketAddress
 
-import spark.Utils
+import org.apache.spark.Utils
 
 
 private[spark] case class ConnectionManagerId(host: String, port: Int) {
diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
similarity index 97%
rename from core/src/main/scala/spark/network/ConnectionManagerTest.scala
rename to core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
index 9e3827aaf56105d5a73415ce5be423a1e598ca03..8d9ad9604d645ffec3232c96150a89f622498237 100644
--- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
-import spark._
-import spark.SparkContext._
+import org.apache.spark._
+import org.apache.spark.SparkContext._
 
 import scala.io.Source
 
diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/org/apache/spark/network/Message.scala
similarity index 98%
rename from core/src/main/scala/spark/network/Message.scala
rename to core/src/main/scala/org/apache/spark/network/Message.scala
index a25457ea35ff76a52f5f5247caf36f5b4c413987..f2ecc6d439aaad826578cb4f5c86d4ada559ee07 100644
--- a/core/src/main/scala/spark/network/Message.scala
+++ b/core/src/main/scala/org/apache/spark/network/Message.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
 import java.nio.ByteBuffer
 import java.net.InetSocketAddress
diff --git a/core/src/main/scala/spark/network/MessageChunk.scala b/core/src/main/scala/org/apache/spark/network/MessageChunk.scala
similarity index 97%
rename from core/src/main/scala/spark/network/MessageChunk.scala
rename to core/src/main/scala/org/apache/spark/network/MessageChunk.scala
index 784db5ab62dce219e8f8fc99c628616954ad2b32..e0fe57b80d5cd693080f9e7e79fb5a8a4f01f569 100644
--- a/core/src/main/scala/spark/network/MessageChunk.scala
+++ b/core/src/main/scala/org/apache/spark/network/MessageChunk.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
 import java.nio.ByteBuffer
 
diff --git a/core/src/main/scala/spark/network/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala
similarity index 98%
rename from core/src/main/scala/spark/network/MessageChunkHeader.scala
rename to core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala
index 18d0cbcc14642eb65ef1a0fcd518207299728349..235fbc39b3bd254e5457652378abf3f1c956377a 100644
--- a/core/src/main/scala/spark/network/MessageChunkHeader.scala
+++ b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
 import java.net.InetAddress
 import java.net.InetSocketAddress
diff --git a/core/src/main/scala/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
similarity index 97%
rename from core/src/main/scala/spark/network/ReceiverTest.scala
rename to core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
index 2bbc736f4055ab0abdc5a8815009092a785de0d3..781715108be6361197eed6e946d3a6664c5161b7 100644
--- a/core/src/main/scala/spark/network/ReceiverTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
 import java.nio.ByteBuffer
 import java.net.InetAddress
diff --git a/core/src/main/scala/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala
similarity index 98%
rename from core/src/main/scala/spark/network/SenderTest.scala
rename to core/src/main/scala/org/apache/spark/network/SenderTest.scala
index 542c54c36b0e7dd0ef1a81abf27da9462a5a41d8..777574980fbc63a82ebc51b60ab02fa7a37c5c6e 100644
--- a/core/src/main/scala/spark/network/SenderTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network
+package org.apache.spark.network
 
 import java.nio.ByteBuffer
 import java.net.InetAddress
diff --git a/core/src/main/scala/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala
similarity index 96%
rename from core/src/main/scala/spark/network/netty/FileHeader.scala
rename to core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala
index bf46d32aa3a473c46b1d8fc41ab209eb57c89678..3c29700920a341cd69fed842f12322098092989b 100644
--- a/core/src/main/scala/spark/network/netty/FileHeader.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.network.netty
+package org.apache.spark.network.netty
 
 import io.netty.buffer._
 
-import spark.Logging
+import org.apache.spark.Logging
 
 private[spark] class FileHeader (
   val fileLen: Int,
diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
similarity index 96%
rename from core/src/main/scala/spark/network/netty/ShuffleCopier.scala
rename to core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
index b01f6369f659c7ce87ccd04ff4760d4c8d0fd1c5..9493ccffd99e52b230326bb29d5d24b8139f29fb 100644
--- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.network.netty
+package org.apache.spark.network.netty
 
 import java.util.concurrent.Executors
 
@@ -23,8 +23,8 @@ import io.netty.buffer.ByteBuf
 import io.netty.channel.ChannelHandlerContext
 import io.netty.util.CharsetUtil
 
-import spark.Logging
-import spark.network.ConnectionManagerId
+import org.apache.spark.Logging
+import org.apache.spark.network.ConnectionManagerId
 
 import scala.collection.JavaConverters._
 
diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala
similarity index 96%
rename from core/src/main/scala/spark/network/netty/ShuffleSender.scala
rename to core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala
index cdf88b03a0840ce884d65543ccf01cecd35b4ef7..537f225469a8df772886ea315365e05e810726f2 100644
--- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.network.netty
+package org.apache.spark.network.netty
 
 import java.io.File
 
-import spark.Logging
+import org.apache.spark.Logging
 
 
 private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging {
diff --git a/core/src/main/scala/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala
similarity index 65%
rename from core/src/main/scala/spark/package.scala
rename to core/src/main/scala/org/apache/spark/package.scala
index b244bfbf069b71affb990159baa44cbd8f2d06f3..112648068909097bfc366b88c31631fe5944276a 100644
--- a/core/src/main/scala/spark/package.scala
+++ b/core/src/main/scala/org/apache/spark/package.scala
@@ -16,16 +16,16 @@
  */
 
 /**
- * Core Spark functionality. [[spark.SparkContext]] serves as the main entry point to Spark, while
- * [[spark.RDD]] is the data type representing a distributed collection, and provides most
+ * Core Spark functionality. [[org.apache.spark.SparkContext]] serves as the main entry point to Spark, while
+ * [[org.apache.spark.RDD]] is the data type representing a distributed collection, and provides most
  * parallel operations. 
  *
- * In addition, [[spark.PairRDDFunctions]] contains operations available only on RDDs of key-value
- * pairs, such as `groupByKey` and `join`; [[spark.DoubleRDDFunctions]] contains operations
- * available only on RDDs of Doubles; and [[spark.SequenceFileRDDFunctions]] contains operations
+ * In addition, [[org.apache.spark.PairRDDFunctions]] contains operations available only on RDDs of key-value
+ * pairs, such as `groupByKey` and `join`; [[org.apache.spark.DoubleRDDFunctions]] contains operations
+ * available only on RDDs of Doubles; and [[org.apache.spark.SequenceFileRDDFunctions]] contains operations
  * available on RDDs that can be saved as SequenceFiles. These operations are automatically
  * available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit conversions when
- * you `import spark.SparkContext._`.
+ * you `import org.apache.spark.SparkContext._`.
  */
 package object spark { 
   // For package docs only
diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala
similarity index 96%
rename from core/src/main/scala/spark/partial/ApproximateActionListener.scala
rename to core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala
index 691d939150a4d14354cf140575138d6928e11d1b..c5d51bee507d426338fac2f7a40686aff3423f05 100644
--- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala
+++ b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
-import spark._
-import spark.scheduler.JobListener
+import org.apache.spark._
+import org.apache.spark.scheduler.JobListener
 
 /**
  * A JobListener for an approximate single-result action, such as count() or non-parallel reduce().
diff --git a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/ApproximateEvaluator.scala
similarity index 97%
rename from core/src/main/scala/spark/partial/ApproximateEvaluator.scala
rename to core/src/main/scala/org/apache/spark/partial/ApproximateEvaluator.scala
index 5eae144dfbccc575a9f973c4547099057dfda18c..9c2859c8b9850295aae46f32387e9f6b81128369 100644
--- a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala
+++ b/core/src/main/scala/org/apache/spark/partial/ApproximateEvaluator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 /**
  * An object that computes a function incrementally by merging in results of type U from multiple
diff --git a/core/src/main/scala/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala
similarity index 96%
rename from core/src/main/scala/spark/partial/BoundedDouble.scala
rename to core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala
index 8bdbe6c0123e712ab1a36796b25ffcb9938c799e..5f4450859cc9b2c6810484bfc995cd7aa32af710 100644
--- a/core/src/main/scala/spark/partial/BoundedDouble.scala
+++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 /**
  * A Double with error bars on it.
diff --git a/core/src/main/scala/spark/partial/CountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala
similarity index 98%
rename from core/src/main/scala/spark/partial/CountEvaluator.scala
rename to core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala
index 6aa92094eb4a0f71911f2cfab5eea744e25876d6..3155dfe165664ff9e006a058585dfef6d20e1417 100644
--- a/core/src/main/scala/spark/partial/CountEvaluator.scala
+++ b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 import cern.jet.stat.Probability
 
diff --git a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala
similarity index 98%
rename from core/src/main/scala/spark/partial/GroupedCountEvaluator.scala
rename to core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala
index ebe2e5a1e3aff9d80b29809e5bffe8147c89229a..e519e3a54846e4b609f319cdb9c508db81542551 100644
--- a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala
+++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 import java.util.{HashMap => JHashMap}
 import java.util.{Map => JMap}
diff --git a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala
similarity index 97%
rename from core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala
rename to core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala
index 2dadbbd5fb49628d16f4614525006cb2cc59eebf..cf8a5680b663d0663f306d3a88eae72e25c5c968 100644
--- a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala
+++ b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 import java.util.{HashMap => JHashMap}
 import java.util.{Map => JMap}
@@ -24,7 +24,7 @@ import scala.collection.mutable.HashMap
 import scala.collection.Map
 import scala.collection.JavaConversions.mapAsScalaMap
 
-import spark.util.StatCounter
+import org.apache.spark.util.StatCounter
 
 /**
  * An ApproximateEvaluator for means by key. Returns a map of key to confidence interval.
diff --git a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala
similarity index 97%
rename from core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
rename to core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala
index ae2b63f7cb7dea855481e6454d0e2e729ceb84a2..8225a5d933ce50261db5edb4bb8c372d1915068f 100644
--- a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
+++ b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 import java.util.{HashMap => JHashMap}
 import java.util.{Map => JMap}
@@ -24,7 +24,7 @@ import scala.collection.mutable.HashMap
 import scala.collection.Map
 import scala.collection.JavaConversions.mapAsScalaMap
 
-import spark.util.StatCounter
+import org.apache.spark.util.StatCounter
 
 /**
  * An ApproximateEvaluator for sums by key. Returns a map of key to confidence interval.
diff --git a/core/src/main/scala/spark/partial/MeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala
similarity index 96%
rename from core/src/main/scala/spark/partial/MeanEvaluator.scala
rename to core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala
index 5ddcad70759eb048d7157f103da2b81cf639757e..d24959cba8727016c23eb87d7444d44e79f00ce9 100644
--- a/core/src/main/scala/spark/partial/MeanEvaluator.scala
+++ b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 import cern.jet.stat.Probability
 
-import spark.util.StatCounter
+import org.apache.spark.util.StatCounter
 
 /**
  * An ApproximateEvaluator for means.
diff --git a/core/src/main/scala/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala
similarity index 99%
rename from core/src/main/scala/spark/partial/PartialResult.scala
rename to core/src/main/scala/org/apache/spark/partial/PartialResult.scala
index 922a9f9bc65a0c7eaabd273b1e2c54ac9b9c53b3..5ce49b8100ee6ad42afa29b4add097c0bb3f3742 100644
--- a/core/src/main/scala/spark/partial/PartialResult.scala
+++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 class PartialResult[R](initialVal: R, isFinal: Boolean) {
   private var finalValue: Option[R] = if (isFinal) Some(initialVal) else None
diff --git a/core/src/main/scala/spark/partial/StudentTCacher.scala b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala
similarity index 98%
rename from core/src/main/scala/spark/partial/StudentTCacher.scala
rename to core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala
index f3bb987d46c3320c261488e0166cfc466d24008f..92915ee66d29f3a2100a6aa3ffb12fa435dffd74 100644
--- a/core/src/main/scala/spark/partial/StudentTCacher.scala
+++ b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 import cern.jet.stat.Probability
 
diff --git a/core/src/main/scala/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala
similarity index 97%
rename from core/src/main/scala/spark/partial/SumEvaluator.scala
rename to core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala
index 4083abef03d25482ae18fd7e1dcdcfe873855505..a74f80094434c02cbb00a5e25cc7db3b6558e620 100644
--- a/core/src/main/scala/spark/partial/SumEvaluator.scala
+++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.partial
+package org.apache.spark.partial
 
 import cern.jet.stat.Probability
 
-import spark.util.StatCounter
+import org.apache.spark.util.StatCounter
 
 /**
  * An ApproximateEvaluator for sums. It estimates the mean and the cont and multiplies them
diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
similarity index 92%
rename from core/src/main/scala/spark/rdd/BlockRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
index 03800584ae0a012edea90042904d86380fd6f020..4bb01efa8650a1332707d78b53059208a80a5bc4 100644
--- a/core/src/main/scala/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}
-import spark.storage.BlockManager
+import org.apache.spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}
+import org.apache.spark.storage.BlockManager
 
 private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition {
   val index = idx
diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
similarity index 98%
rename from core/src/main/scala/spark/rdd/CartesianRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
index 91b3e69d6fb3e759916c917be1d4e8b004466b53..9b0c882481cfc8357e3758dc89c7197ae2dd648c 100644
--- a/core/src/main/scala/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import java.io.{ObjectOutputStream, IOException}
-import spark._
+import org.apache.spark._
 
 
 private[spark]
diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
similarity index 98%
rename from core/src/main/scala/spark/rdd/CheckpointRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
index 1ad5fe6539017fb0e2a9a1dd7b598758da03faf3..3311757189aeebe874c048345ce880eae23bd316 100644
--- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark._
+import org.apache.spark._
 import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter}
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.{NullWritable, BytesWritable}
@@ -138,7 +138,7 @@ private[spark] object CheckpointRDD extends Logging {
   // each split file having multiple blocks. This needs to be run on a
   // cluster (mesos or standalone) using HDFS.
   def main(args: Array[String]) {
-    import spark._
+    import org.apache.spark._
 
     val Array(cluster, hdfsPath) = args
     val env = SparkEnv.get
diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
similarity index 96%
rename from core/src/main/scala/spark/rdd/CoGroupedRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index 01b6c23dcc202a5b701dbbf34d97aa529a0eaeec..dcc35e8d0e9c4961f9be5c36cc36d54806ee40cc 100644
--- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import java.io.{ObjectOutputStream, IOException}
 import java.util.{HashMap => JHashMap}
@@ -23,8 +23,8 @@ import java.util.{HashMap => JHashMap}
 import scala.collection.JavaConversions
 import scala.collection.mutable.ArrayBuffer
 
-import spark.{Partition, Partitioner, RDD, SparkEnv, TaskContext}
-import spark.{Dependency, OneToOneDependency, ShuffleDependency}
+import org.apache.spark.{Partition, Partitioner, RDD, SparkEnv, TaskContext}
+import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency}
 
 
 private[spark] sealed trait CoGroupSplitDep extends Serializable
diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
similarity index 99%
rename from core/src/main/scala/spark/rdd/CoalescedRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
index e612d026b221a1a7ccc8e9013639c93ffbee4a4b..c5de6362a9aa7c75298b3e38029e5ab8a4e43388 100644
--- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark._
+import org.apache.spark._
 import java.io.{ObjectOutputStream, IOException}
 import scala.collection.mutable
 import scala.Some
diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala
similarity index 91%
rename from core/src/main/scala/spark/rdd/EmptyRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala
index d7d4db5d3093642fc1258b55621a1f71f032d77e..24ce4abbc47b519f6cac7ea43ba08f2da3d3d305 100644
--- a/core/src/main/scala/spark/rdd/EmptyRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}
+import org.apache.spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}
 
 
 /**
diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala
similarity index 92%
rename from core/src/main/scala/spark/rdd/FilteredRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala
index 783508cfd173ca6441760c6cfc93c7f3d0562817..4df8ceb58bee0013513ba5a30b28cda0d698b093 100644
--- a/core/src/main/scala/spark/rdd/FilteredRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{OneToOneDependency, RDD, Partition, TaskContext}
+import org.apache.spark.{OneToOneDependency, RDD, Partition, TaskContext}
 
 private[spark] class FilteredRDD[T: ClassManifest](
     prev: RDD[T],
diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala
similarity index 93%
rename from core/src/main/scala/spark/rdd/FlatMappedRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala
index ed75eac3ff6edba36181a789167bca2c2484db70..2bf7653af1b2635c4c59483f653d97291f9d8869 100644
--- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{RDD, Partition, TaskContext}
+import org.apache.spark.{RDD, Partition, TaskContext}
 
 
 private[spark]
diff --git a/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala
similarity index 93%
rename from core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala
index a6bdce89d8a4ce317107e31da8d11ba8b14cfb2d..e544720b0546b71f74e089fa936e1fe3f83974c5 100644
--- a/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{TaskContext, Partition, RDD}
+import org.apache.spark.{TaskContext, Partition, RDD}
 
 
 private[spark]
diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala
similarity index 93%
rename from core/src/main/scala/spark/rdd/GlommedRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala
index 1573f8a289a697499004d8b13d4f0ae3a629a335..2ce94199f2e41a7de9891a777dfb4c310cf3169e 100644
--- a/core/src/main/scala/spark/rdd/GlommedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{RDD, Partition, TaskContext}
+import org.apache.spark.{RDD, Partition, TaskContext}
 
 private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T])
   extends RDD[Array[T]](prev) {
diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
similarity index 95%
rename from core/src/main/scala/spark/rdd/HadoopRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index e512423fd63ab464de4ab3711e300d1a5726b3a4..08e6154bb9ff075d0e4bdfe33ce021a26b28c5ab 100644
--- a/core/src/main/scala/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import java.io.EOFException
 import java.util.NoSuchElementException
@@ -32,8 +32,8 @@ import org.apache.hadoop.mapred.RecordReader
 import org.apache.hadoop.mapred.Reporter
 import org.apache.hadoop.util.ReflectionUtils
 
-import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext}
-import spark.util.NextIterator
+import org.apache.spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext}
+import org.apache.spark.util.NextIterator
 import org.apache.hadoop.conf.{Configuration, Configurable}
 
 
diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
similarity index 96%
rename from core/src/main/scala/spark/rdd/JdbcRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index 59132437d20cf817343c326e9a65ab1aeaeba79f..3db460b3ced5f025839c24a840b1b8871baa6fe7 100644
--- a/core/src/main/scala/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import java.sql.{Connection, ResultSet}
 
-import spark.{Logging, Partition, RDD, SparkContext, TaskContext}
-import spark.util.NextIterator
+import org.apache.spark.{Logging, Partition, RDD, SparkContext, TaskContext}
+import org.apache.spark.util.NextIterator
 
 private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition {
   override def index = idx
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
similarity index 93%
rename from core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
index af8f0a112f2acd184af1211faa04119adeab3e1e..13009d3e17227b9d505b714c3341a4bd3560d61f 100644
--- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{RDD, Partition, TaskContext}
+import org.apache.spark.{RDD, Partition, TaskContext}
 
 
 private[spark]
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala
similarity index 94%
rename from core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala
index 3b4e9518fd0acc2706cffa20e1771ea72bc746ba..1683050b865d794e53b8acd904de7a13ab596553 100644
--- a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{RDD, Partition, TaskContext}
+import org.apache.spark.{RDD, Partition, TaskContext}
 
 
 /**
diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
similarity index 93%
rename from core/src/main/scala/spark/rdd/MappedRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
index 8b411dd85d9159465893b97067b7919d54f46cdb..26d4806edb0208abd17cc3531a066c2dbc342b5e 100644
--- a/core/src/main/scala/spark/rdd/MappedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{RDD, Partition, TaskContext}
+import org.apache.spark.{RDD, Partition, TaskContext}
 
 private[spark]
 class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U)
diff --git a/core/src/main/scala/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala
similarity index 93%
rename from core/src/main/scala/spark/rdd/MappedValuesRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala
index 8334e3b557dd22f8deb597b8e11c775a0d4dee63..a405e9acddbf2199c24481a75387ee76305e4b60 100644
--- a/core/src/main/scala/spark/rdd/MappedValuesRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 
-import spark.{TaskContext, Partition, RDD}
+import org.apache.spark.{TaskContext, Partition, RDD}
 
 private[spark]
 class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U)
diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
similarity index 96%
rename from core/src/main/scala/spark/rdd/NewHadoopRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index b1877dc06e5103ca6b57099a35d546b973a3cd30..114b5044861ac6268032dfe26527a1058c9d64ab 100644
--- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import java.text.SimpleDateFormat
 import java.util.Date
@@ -24,7 +24,7 @@ import org.apache.hadoop.conf.{Configurable, Configuration}
 import org.apache.hadoop.io.Writable
 import org.apache.hadoop.mapreduce._
 
-import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
+import org.apache.spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
 
 
 private[spark]
diff --git a/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
similarity index 96%
rename from core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala
rename to core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
index 9154b760359aa1836d214219f216a136dfc25b09..4c3df0eaf46cd03d2573edba5793910dfd450dbd 100644
--- a/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{RangePartitioner, Logging, RDD}
+import org.apache.spark.{RangePartitioner, Logging, RDD}
 
 /**
  * Extra functions available on RDDs of (key, value) pairs where the key is sortable through
diff --git a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
similarity index 98%
rename from core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
index 33079cd53937d99062c2fd6ddc0e04a68476a007..8db3611054d62c521e92208425b4aad31c1e6cc0 100644
--- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import scala.collection.immutable.NumericRange
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.Map
-import spark._
+import org.apache.spark._
 import java.io._
 import scala.Serializable
 
diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
similarity index 96%
rename from core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
index d8700becb0e870ff5242c77219b53c777771c210..8e79a5c874eb97966dd9db1209e59c351f6aeb14 100644
--- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext}
+import org.apache.spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext}
 
 
 class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition {
diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
similarity index 96%
rename from core/src/main/scala/spark/rdd/PipedRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
index 2cefdc78b01b27da5f051c6d75ae59bc685dfac7..98498d5ddfe1b46f104d3e6a906d8bd8bbe37c13 100644
--- a/core/src/main/scala/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import java.io.PrintWriter
 import java.util.StringTokenizer
@@ -25,8 +25,8 @@ import scala.collection.JavaConversions._
 import scala.collection.mutable.ArrayBuffer
 import scala.io.Source
 
-import spark.{RDD, SparkEnv, Partition, TaskContext}
-import spark.broadcast.Broadcast
+import org.apache.spark.{RDD, SparkEnv, Partition, TaskContext}
+import org.apache.spark.broadcast.Broadcast
 
 
 /**
diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
similarity index 96%
rename from core/src/main/scala/spark/rdd/SampledRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
index 574c9b141de3f869b1817799679a87f5e279a08a..1e8d89e912063f569d01e4f000a3b8307c175880 100644
--- a/core/src/main/scala/spark/rdd/SampledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import java.util.Random
 
 import cern.jet.random.Poisson
 import cern.jet.random.engine.DRand
 
-import spark.{RDD, Partition, TaskContext}
+import org.apache.spark.{RDD, Partition, TaskContext}
 
 private[spark]
 class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable {
diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
similarity index 94%
rename from core/src/main/scala/spark/rdd/ShuffledRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
index 51c05af064ef7456c0020d0f78a68ebd31a9b069..f0e9ab8b801e612ecf39952c4b08ffc0811fc492 100644
--- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{Dependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext}
+import org.apache.spark.{Dependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext}
 
 
 private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
similarity index 93%
rename from core/src/main/scala/spark/rdd/SubtractedRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
index dadef5e17de594431ca70002438b025448248cfc..7369dfaa74b1921a4078267632e840abcf770c13 100644
--- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
@@ -15,19 +15,19 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import java.util.{HashMap => JHashMap}
 import scala.collection.JavaConversions._
 import scala.collection.mutable.ArrayBuffer
-import spark.RDD
-import spark.Partitioner
-import spark.Dependency
-import spark.TaskContext
-import spark.Partition
-import spark.SparkEnv
-import spark.ShuffleDependency
-import spark.OneToOneDependency
+import org.apache.spark.RDD
+import org.apache.spark.Partitioner
+import org.apache.spark.Dependency
+import org.apache.spark.TaskContext
+import org.apache.spark.Partition
+import org.apache.spark.SparkEnv
+import org.apache.spark.ShuffleDependency
+import org.apache.spark.OneToOneDependency
 
 
 /**
diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
similarity index 95%
rename from core/src/main/scala/spark/rdd/UnionRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
index 2776826f187aab0e2d5f6cdb4c6adaef9231215e..fd02476b627b7b32406488eed8264cd4bf4d427b 100644
--- a/core/src/main/scala/spark/rdd/UnionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import scala.collection.mutable.ArrayBuffer
-import spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext}
+import org.apache.spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext}
 import java.io.{ObjectOutputStream, IOException}
 
 private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int)
diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
similarity index 97%
rename from core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
index 9a0831bd899e2ff48e7db6601e8fdc72ac279208..5ae1db3e6761ff6e23c41a51d7dead1d21278c87 100644
--- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext}
+import org.apache.spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext}
 import java.io.{ObjectOutputStream, IOException}
 
 private[spark] class ZippedPartitionsPartition(
diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala
similarity index 95%
rename from core/src/main/scala/spark/rdd/ZippedRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala
index 4074e50e44040b96b756efeb69c238b7165ff142..3bd00d291b173191204edea924089dd9ead53b8a 100644
--- a/core/src/main/scala/spark/rdd/ZippedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
-import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext}
+import org.apache.spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext}
 import java.io.{ObjectOutputStream, IOException}
 
 
diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala
similarity index 94%
rename from core/src/main/scala/spark/scheduler/ActiveJob.scala
rename to core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala
index fecc3e9648a4ab749857c9dd371df4d3e98a8b64..0b04607d019a8431b1ab1aa07f3bda77f3421151 100644
--- a/core/src/main/scala/spark/scheduler/ActiveJob.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
-import spark.TaskContext
+import org.apache.spark.TaskContext
 
 import java.util.Properties
 
diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
similarity index 98%
rename from core/src/main/scala/spark/scheduler/DAGScheduler.scala
rename to core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 7275bd346ad33a3a8e5a915b2f2eeadc416996a2..5ac700bbf4408794bbd51e000de7fd550d461dbd 100644
--- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.io.NotSerializableException
 import java.util.Properties
@@ -24,12 +24,12 @@ import java.util.concurrent.atomic.AtomicInteger
 
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map}
 
-import spark._
-import spark.executor.TaskMetrics
-import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult}
-import spark.scheduler.cluster.TaskInfo
-import spark.storage.{BlockManager, BlockManagerMaster}
-import spark.util.{MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark._
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult}
+import org.apache.spark.scheduler.cluster.TaskInfo
+import org.apache.spark.storage.{BlockManager, BlockManagerMaster}
+import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
 
 /**
  * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of
diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
similarity index 93%
rename from core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
rename to core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
index b8ba0e92394c257aa5432253f1e9f6030ba141f1..5b07933eedc1def166c031ae3d92c96e435cf0d3 100644
--- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.util.Properties
 
-import spark.scheduler.cluster.TaskInfo
+import org.apache.spark.scheduler.cluster.TaskInfo
 import scala.collection.mutable.Map
 
-import spark._
-import spark.executor.TaskMetrics
+import org.apache.spark._
+import org.apache.spark.executor.TaskMetrics
 
 /**
  * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue
diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
similarity index 92%
rename from core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala
rename to core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
index 98c4fb7e59a03755cebefb21fe6a99f0a2a89f82..ce0dc9093d8d36e0b8538731db102af7e13879eb 100644
--- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
@@ -1,8 +1,8 @@
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import com.codahale.metrics.{Gauge,MetricRegistry}
 
-import spark.metrics.source.Source
+import org.apache.spark.metrics.source.Source
 
 private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source {
   val metricRegistry = new MetricRegistry()
diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
similarity index 98%
rename from core/src/main/scala/spark/scheduler/InputFormatInfo.scala
rename to core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
index 8f1b9b29b54c3f8c309b9f09fbe827fb9eb20b51..370ccd183c37849ce269798d9aa4a0a891368343 100644
--- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
-import spark.{Logging, SparkEnv}
+import org.apache.spark.{Logging, SparkEnv}
 import scala.collection.immutable.Set
 import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
 import org.apache.hadoop.security.UserGroupInformation
diff --git a/core/src/main/scala/spark/scheduler/JobListener.scala b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala
similarity index 97%
rename from core/src/main/scala/spark/scheduler/JobListener.scala
rename to core/src/main/scala/org/apache/spark/scheduler/JobListener.scala
index af108b8fec35674f5535fa4a37e999403059555d..50c2b9acd609c9b781b1f95a0a0547da50ea035b 100644
--- a/core/src/main/scala/spark/scheduler/JobListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 /**
  * Interface used to listen for job completion or failure events after submitting a job to the
diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
similarity index 96%
rename from core/src/main/scala/spark/scheduler/JobLogger.scala
rename to core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
index 1bc9fabdffdcb83eda48fa341eb87a9dd90f9f0b..98ef4d1e634f4ecd29ed951041f3fd1293e68e37 100644
--- a/core/src/main/scala/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.io.PrintWriter
 import java.io.File
@@ -27,9 +27,9 @@ import java.util.concurrent.LinkedBlockingQueue
 import scala.collection.mutable.{Map, HashMap, ListBuffer}
 import scala.io.Source
 
-import spark._
-import spark.executor.TaskMetrics
-import spark.scheduler.cluster.TaskInfo
+import org.apache.spark._
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.scheduler.cluster.TaskInfo
 
 // Used to record runtime information for each job, including RDD graph 
 // tasks' start/stop shuffle information and information from outside
diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala
similarity index 96%
rename from core/src/main/scala/spark/scheduler/JobResult.scala
rename to core/src/main/scala/org/apache/spark/scheduler/JobResult.scala
index a61b335152ebb700d2dfd1abe4e6482f9e59be02..c381348a8d424af31171f1d8d07ab7fb931afc8d 100644
--- a/core/src/main/scala/spark/scheduler/JobResult.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 /**
  * A result of a job in the DAGScheduler.
diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
similarity index 98%
rename from core/src/main/scala/spark/scheduler/JobWaiter.scala
rename to core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
index 69cd161c1f0eba0dcd68b23c629464e3de4ac00a..200d8817990a42a7de74245a4d66a7b148778941 100644
--- a/core/src/main/scala/spark/scheduler/JobWaiter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import scala.collection.mutable.ArrayBuffer
 
diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
similarity index 95%
rename from core/src/main/scala/spark/scheduler/MapStatus.scala
rename to core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
index 2f6a68ee85b248a378e74c0e69fe3bd8420b6d28..1c61687f280064998b704c4d96422af4e5cd2057 100644
--- a/core/src/main/scala/spark/scheduler/MapStatus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
-import spark.storage.BlockManagerId
+import org.apache.spark.storage.BlockManagerId
 import java.io.{ObjectOutput, ObjectInput, Externalizable}
 
 /**
diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
similarity index 98%
rename from core/src/main/scala/spark/scheduler/ResultTask.scala
rename to core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index d066df5dc1dddfc288efc1deed9610bda6d30d9b..2f157ccdd25a8893fdcb000f974f242e5c125fce 100644
--- a/core/src/main/scala/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
-import spark._
+import org.apache.spark._
 import java.io._
 import util.{MetadataCleaner, TimeStampedHashMap}
 import java.util.zip.{GZIPInputStream, GZIPOutputStream}
diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
similarity index 96%
rename from core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
rename to core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index f2a038576b722cce8c8d2737d7856eeef8091c32..ca716b44e885ce608233205202553b8345ae6877 100644
--- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.io._
 import java.util.zip.{GZIPInputStream, GZIPOutputStream}
 
 import scala.collection.mutable.HashMap
 
-import spark._
-import spark.executor.ShuffleWriteMetrics
-import spark.storage._
-import spark.util.{TimeStampedHashMap, MetadataCleaner}
+import org.apache.spark._
+import org.apache.spark.executor.ShuffleWriteMetrics
+import org.apache.spark.storage._
+import org.apache.spark.util.{TimeStampedHashMap, MetadataCleaner}
 
 
 private[spark] object ShuffleMapTask {
diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
similarity index 96%
rename from core/src/main/scala/spark/scheduler/SparkListener.scala
rename to core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index e5531011c2b87f71762cc8a4276cade0deb2c797..3504424fa947577add606c7e91971e4c8e1c9fb7 100644
--- a/core/src/main/scala/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.util.Properties
-import spark.scheduler.cluster.TaskInfo
-import spark.util.Distribution
-import spark.{Logging, SparkContext, TaskEndReason, Utils}
-import spark.executor.TaskMetrics
+import org.apache.spark.scheduler.cluster.TaskInfo
+import org.apache.spark.util.Distribution
+import org.apache.spark.{Logging, SparkContext, TaskEndReason, Utils}
+import org.apache.spark.executor.TaskMetrics
 
 sealed trait SparkListenerEvents
 
@@ -79,7 +79,7 @@ trait SparkListener {
  */
 class StatsReportListener extends SparkListener with Logging {
   override def onStageCompleted(stageCompleted: StageCompleted) {
-    import spark.scheduler.StatsReportListener._
+    import org.apache.spark.scheduler.StatsReportListener._
     implicit val sc = stageCompleted
     this.logInfo("Finished stage: " + stageCompleted.stageInfo)
     showMillisDistribution("task runtime:", (info, _) => Some(info.duration))
diff --git a/core/src/main/scala/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
similarity index 95%
rename from core/src/main/scala/spark/scheduler/SparkListenerBus.scala
rename to core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
index f55ed455ed3f700cc5e6cc7664c1a7d39d40699b..a65e1ecd6d4bdc89903779fa3d12ce82782cdba4 100644
--- a/core/src/main/scala/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.util.concurrent.LinkedBlockingQueue
 
 import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
 
-import spark.Logging
+import org.apache.spark.Logging
 
 /** Asynchronously passes SparkListenerEvents to registered SparkListeners. */
 private[spark] class SparkListenerBus() extends Logging {
diff --git a/core/src/main/scala/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala
similarity index 98%
rename from core/src/main/scala/spark/scheduler/SplitInfo.scala
rename to core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala
index 4e3661ec5da9b71f58ca1a0797885472481b9325..5b40a3eb29b30d57f10c380f8d6ca90dafc8642e 100644
--- a/core/src/main/scala/spark/scheduler/SplitInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import collection.mutable.ArrayBuffer
 
diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
similarity index 97%
rename from core/src/main/scala/spark/scheduler/Stage.scala
rename to core/src/main/scala/org/apache/spark/scheduler/Stage.scala
index c599c00ac4406cb7e91357f4a3c0eb6dbc24eda1..87b1fe4e0c9ff12d9b3437897aa169de944cb661 100644
--- a/core/src/main/scala/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.net.URI
 
-import spark._
-import spark.storage.BlockManagerId
+import org.apache.spark._
+import org.apache.spark.storage.BlockManagerId
 
 /**
  * A stage is a set of independent tasks all computing the same function that need to run as part
diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
similarity index 88%
rename from core/src/main/scala/spark/scheduler/StageInfo.scala
rename to core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index c4026f995a49fd3849796bc389f407b4f88760ef..72cb1c9ce8a2fac9e184db8a30d8cb69d976fc76 100644
--- a/core/src/main/scala/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
-import spark.scheduler.cluster.TaskInfo
+import org.apache.spark.scheduler.cluster.TaskInfo
 import scala.collection._
-import spark.executor.TaskMetrics
+import org.apache.spark.executor.TaskMetrics
 
 case class StageInfo(
     val stage: Stage,
diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
similarity index 95%
rename from core/src/main/scala/spark/scheduler/Task.scala
rename to core/src/main/scala/org/apache/spark/scheduler/Task.scala
index 0ab2ae6cfe9f655ea74420a693f1938c8352beea..598d91752a31e81b715cb324c0e348e8aedd518f 100644
--- a/core/src/main/scala/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
-import spark.serializer.SerializerInstance
+import org.apache.spark.serializer.SerializerInstance
 import java.io.{DataInputStream, DataOutputStream}
 import java.nio.ByteBuffer
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
-import spark.util.ByteBufferInputStream
+import org.apache.spark.util.ByteBufferInputStream
 import scala.collection.mutable.HashMap
-import spark.executor.TaskMetrics
+import org.apache.spark.executor.TaskMetrics
 
 /**
  * A task to execute on a worker node.
diff --git a/core/src/main/scala/spark/scheduler/TaskLocation.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala
similarity index 97%
rename from core/src/main/scala/spark/scheduler/TaskLocation.scala
rename to core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala
index fea117e956d454e7eaa1bea3aa2c7506a6423f08..67c9a6760b1b3766de7730e90b3c371529a646ce 100644
--- a/core/src/main/scala/spark/scheduler/TaskLocation.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 /**
  * A location where a task should run. This can either be a host or a (host, executorID) pair.
diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
similarity index 94%
rename from core/src/main/scala/spark/scheduler/TaskResult.scala
rename to core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
index fc4856756b127d4cee32e34231c4b55365b15f54..776675d28c9a43845c033f99c143813d27de3442 100644
--- a/core/src/main/scala/spark/scheduler/TaskResult.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.io._
 
 import scala.collection.mutable.Map
-import spark.executor.TaskMetrics
-import spark.{Utils, SparkEnv}
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.{Utils, SparkEnv}
 import java.nio.ByteBuffer
 
 // Task result. Also contains updates to accumulator variables.
diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
similarity index 92%
rename from core/src/main/scala/spark/scheduler/TaskScheduler.scala
rename to core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
index 4943d58e254034f43c2f4ce6b305455b5cf40e1a..63be8ba3f58ee584d7626c7113d3162ff61b251b 100644
--- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
-import spark.scheduler.cluster.Pool
-import spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.scheduler.cluster.Pool
+import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
 /**
  * Low-level task scheduler interface, implemented by both ClusterScheduler and LocalScheduler.
  * These schedulers get sets of tasks submitted to them from the DAGScheduler for each stage,
diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala
similarity index 90%
rename from core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
rename to core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala
index 64be50b2d02b8e13b2795024c933379f5312830c..83be051c1a8092029f6bf13b2408ae77fb9fa75f 100644
--- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
-import spark.scheduler.cluster.TaskInfo
+import org.apache.spark.scheduler.cluster.TaskInfo
 import scala.collection.mutable.Map
 
-import spark.TaskEndReason
-import spark.executor.TaskMetrics
+import org.apache.spark.TaskEndReason
+import org.apache.spark.executor.TaskMetrics
 
 /**
  * Interface for getting events back from the TaskScheduler.
diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
similarity index 97%
rename from core/src/main/scala/spark/scheduler/TaskSet.scala
rename to core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
index dc3550dd0b1f705a701757355bdb44cf712db370..c3ad325156f53786224a58fd48e50697c9c94344 100644
--- a/core/src/main/scala/spark/scheduler/TaskSet.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.util.Properties
 
diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
similarity index 98%
rename from core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
index 679d899b472a280805f29abcb901548ddef253c8..3196ab5022aea1fe0ecf2190e4b71a22e0489aee 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import java.lang.{Boolean => JBoolean}
 
@@ -23,10 +23,10 @@ import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
 
-import spark._
-import spark.TaskState.TaskState
-import spark.scheduler._
-import spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark._
+import org.apache.spark.TaskState.TaskState
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
 import java.nio.ByteBuffer
 import java.util.concurrent.atomic.AtomicLong
 import java.util.{TimerTask, Timer}
diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
similarity index 98%
rename from core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
index a4d6880abb55d7b6d2c35c98fb6574a31c45439c..a33307b83ae5c87ba4222e12d8967b267c5f6e32 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import java.nio.ByteBuffer
 import java.util.{Arrays, NoSuchElementException}
@@ -26,15 +26,15 @@ import scala.collection.mutable.HashSet
 import scala.math.max
 import scala.math.min
 
-import spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils}
-import spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure}
-import spark.TaskState.TaskState
-import spark.scheduler._
+import org.apache.spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils}
+import org.apache.spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure}
+import org.apache.spark.TaskState.TaskState
+import org.apache.spark.scheduler._
 import scala.Some
-import spark.FetchFailed
-import spark.ExceptionFailure
-import spark.TaskResultTooBigFailure
-import spark.util.{SystemClock, Clock}
+import org.apache.spark.FetchFailed
+import org.apache.spark.ExceptionFailure
+import org.apache.spark.TaskResultTooBigFailure
+import org.apache.spark.util.{SystemClock, Clock}
 
 
 /**
diff --git a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala
similarity index 93%
rename from core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala
index 8825f2dd2424887dbf91d64a5a447a0853f11715..5077b2b48b5749aaf0b6264ffc93767f5560a9db 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
-import spark.executor.ExecutorExitCode
+import org.apache.spark.executor.ExecutorExitCode
 
 /**
  * Represents an explanation for a executor or whole slave failing or exiting.
diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala
similarity index 96%
rename from core/src/main/scala/spark/scheduler/cluster/Pool.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala
index 83708f07e1a4fb7d77a4dc08380b75803faa57ec..35b32600da6e038fd0aa81bbe3d329527f2fd807 100644
--- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 
-import spark.Logging
-import spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.Logging
+import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
 
 /**
  * An Schedulable entity that represent collection of Pools or TaskSetManagers
diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala
similarity index 93%
rename from core/src/main/scala/spark/scheduler/cluster/Schedulable.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala
index e77e8e4162e1a435d33a6a84117920b71d60ba74..f4726450ec2396e03b58d4f740a945876dba9268 100644
--- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
-import spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
 
 import scala.collection.mutable.ArrayBuffer
 /**
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala
similarity index 97%
rename from core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala
index 2fc8a76a0575b868fad3f076bac068879fafdb42..d04eeb6b98334bd6b9da95622f2741d3d7df2280 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import java.io.{File, FileInputStream, FileOutputStream, FileNotFoundException}
 import java.util.Properties
 
 import scala.xml.XML
 
-import spark.Logging
-import spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.Logging
+import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
 
 
 /**
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
similarity index 93%
rename from core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
index 4431744ec33e838a8fd621474833234d85541250..bde2f73df41041334f4b895324a823006a134bde 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
-import spark.{SparkContext, Utils}
+import org.apache.spark.{SparkContext, Utils}
 
 /**
  * A backend interface for cluster scheduling systems that allows plugging in different ones under
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala
similarity index 98%
rename from core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala
index 69e0ac2a6bd83234b55cdf00894b03c3b4d09d74..cbeed4731a79c3205890beaae31b322085a57369 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 /**
  * An interface for sort algorithm
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala
similarity index 96%
rename from core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala
index 55cdf4791f985725d7bf05f431da100ba281bb60..34811389a02f7d27c6203a1ad6c77cbd90a03a6a 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 /**
  *  "FAIR" and "FIFO" determines which policy is used
diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
similarity index 89%
rename from core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 7ac574bdc8e574e0b852fdffc156fc4a710a27b9..ac6dc7d879360c9a236b73f68d8843f44f1b8859 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
-import spark.{Utils, Logging, SparkContext}
-import spark.deploy.client.{Client, ClientListener}
-import spark.deploy.{Command, ApplicationDescription}
+import org.apache.spark.{Utils, Logging, SparkContext}
+import org.apache.spark.deploy.client.{Client, ClientListener}
+import org.apache.spark.deploy.{Command, ApplicationDescription}
 import scala.collection.mutable.HashMap
 
 private[spark] class SparkDeploySchedulerBackend(
@@ -45,7 +45,8 @@ private[spark] class SparkDeploySchedulerBackend(
       System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
       StandaloneSchedulerBackend.ACTOR_NAME)
     val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
-    val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs)
+    val command = Command(
+      "org.apache.spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs)
     val sparkHome = sc.getSparkHome().getOrElse(null)
     val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
         sc.ui.appUIAddress)
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala
similarity index 92%
rename from core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala
index 05c29eb72f65f573cc1d76334c07cf22f69322c0..1cc5daf67398560dc47ccef8e2d4157c58fb2678 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import java.nio.ByteBuffer
 
-import spark.TaskState.TaskState
-import spark.Utils
-import spark.util.SerializableBuffer
+import org.apache.spark.TaskState.TaskState
+import org.apache.spark.Utils
+import org.apache.spark.util.SerializableBuffer
 
 
 private[spark] sealed trait StandaloneClusterMessage extends Serializable
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
similarity index 97%
rename from core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index 3203be10294d375d860e0e14ed24912a2edb31d0..3677a827e0c05fd817e9dbf5dba813cfd9e88d1a 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import java.util.concurrent.atomic.AtomicInteger
 
@@ -28,8 +28,8 @@ import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClient
 import akka.util.Duration
 import akka.util.duration._
 
-import spark.{Utils, SparkException, Logging, TaskState}
-import spark.scheduler.cluster.StandaloneClusterMessages._
+import org.apache.spark.{Utils, SparkException, Logging, TaskState}
+import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._
 
 /**
  * A standalone scheduler backend, which waits for standalone executors to connect to it through
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala
similarity index 93%
rename from core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala
index 187553233fb42a28ec85c95fc4f8710eccb08c2a..309ac2f6c9d0537bd178570debf2b7c315be59de 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import java.nio.ByteBuffer
-import spark.util.SerializableBuffer
+import org.apache.spark.util.SerializableBuffer
 
 private[spark] class TaskDescription(
     val taskId: Long,
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala
similarity index 96%
rename from core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala
index c2c5522686f74d62384a5acb0d25d50d7e988085..7ce14be7fb9d8a4a5bf7fdad1c0f980888832f20 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
-import spark.Utils
+import org.apache.spark.Utils
 
 /**
  * Information about a running task attempt inside a TaskSet.
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala
similarity index 96%
rename from core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala
index 1c33e41f87e81d082bd3151c4f55fd39df32f32f..5d4130e14a8ad1ab40f87111e56596c7ff217f0d 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 
 private[spark] object TaskLocality
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala
similarity index 93%
rename from core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala
index 0248830b7a24c250dbdd6b41d2c9a4068f692f88..648a3ef922c906e7fbe82f7ade252bd8f6b0f7ca 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import java.nio.ByteBuffer
 
-import spark.TaskState.TaskState
-import spark.scheduler.TaskSet
+import org.apache.spark.TaskState.TaskState
+import org.apache.spark.scheduler.TaskSet
 
 /**
  * Tracks and schedules the tasks within a single TaskSet. This class keeps track of the status of
diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala
similarity index 95%
rename from core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
rename to core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala
index 1d09bd9b034193f71bf22e2aa8bb5c44424c2994..938f62883a104df5e60b11333e939ef20293792e 100644
--- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 /**
  * Represents free resources available on an executor.
diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
similarity index 97%
rename from core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
rename to core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
index 5be4dbd9f0b527489eeba69ac234cbc8b8593ad8..f0ebe66d8239128f3e2ec4eccc3dd3cfb094153d 100644
--- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.local
+package org.apache.spark.scheduler.local
 
 import java.io.File
 import java.lang.management.ManagementFactory
@@ -27,12 +27,12 @@ import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
 
-import spark._
-import spark.TaskState.TaskState
-import spark.executor.ExecutorURLClassLoader
-import spark.scheduler._
-import spark.scheduler.cluster._
-import spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark._
+import org.apache.spark.TaskState.TaskState
+import org.apache.spark.executor.ExecutorURLClassLoader
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster._
+import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
 import akka.actor._
 
 /**
diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
similarity index 94%
rename from core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala
rename to core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
index e237f289e367bd31062ad0e99eff8d7b571ea1b7..e52cb998bdf42df09e45b23b2692c9aa0b497a13 100644
--- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package spark.scheduler.local
+package org.apache.spark.scheduler.local
 
 import java.nio.ByteBuffer
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 
-import spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState}
-import spark.TaskState.TaskState
-import spark.scheduler.{Task, TaskResult, TaskSet}
-import spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager}
+import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState}
+import org.apache.spark.TaskState.TaskState
+import org.apache.spark.scheduler.{Task, TaskResult, TaskSet}
+import org.apache.spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager}
 
 
 private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet)
diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
similarity index 94%
rename from core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
rename to core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
index eef3ee142519f0ac9b6cdf6e62bc576090fa208e..f6a2feab289a6fcf68d2e3b694930ab9d10ff8dd 100644
--- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.mesos
+package org.apache.spark.scheduler.mesos
 
 import com.google.protobuf.ByteString
 
@@ -23,14 +23,14 @@ import org.apache.mesos.{Scheduler => MScheduler}
 import org.apache.mesos._
 import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _}
 
-import spark.{SparkException, Utils, Logging, SparkContext}
+import org.apache.spark.{SparkException, Utils, Logging, SparkContext}
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 import scala.collection.JavaConversions._
 import java.io.File
-import spark.scheduler.cluster._
+import org.apache.spark.scheduler.cluster._
 import java.util.{ArrayList => JArrayList, List => JList}
 import java.util.Collections
-import spark.TaskState
+import org.apache.spark.TaskState
 
 /**
  * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds
@@ -126,14 +126,16 @@ private[spark] class CoarseMesosSchedulerBackend(
     val uri = System.getProperty("spark.executor.uri")
     if (uri == null) {
       val runScript = new File(sparkHome, "spark-class").getCanonicalPath
-      command.setValue("\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
-        runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
+      command.setValue(
+        "\"%s\" org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
+          runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
     } else {
       // Grab everything to the first '.'. We'll use that and '*' to
       // glob the directory "correctly".
       val basename = uri.split('/').last.split('.').head
-      command.setValue("cd %s*; ./spark-class spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
-        basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
+      command.setValue(
+        "cd %s*; ./spark-class org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
+          basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
       command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
     }
     return command.build()
diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala
similarity index 98%
rename from core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
rename to core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala
index f6069a5775e2ba6e3550091ab69ff7dff62f4776..e002af17422ecf1aa8f0a656a55d6c25aa1e7d80 100644
--- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.scheduler.mesos
+package org.apache.spark.scheduler.mesos
 
 import com.google.protobuf.ByteString
 
@@ -23,14 +23,14 @@ import org.apache.mesos.{Scheduler => MScheduler}
 import org.apache.mesos._
 import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _}
 
-import spark.{SparkException, Utils, Logging, SparkContext}
+import org.apache.spark.{SparkException, Utils, Logging, SparkContext}
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 import scala.collection.JavaConversions._
 import java.io.File
-import spark.scheduler.cluster._
+import org.apache.spark.scheduler.cluster._
 import java.util.{ArrayList => JArrayList, List => JList}
 import java.util.Collections
-import spark.TaskState
+import org.apache.spark.TaskState
 
 /**
  * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a
diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
similarity index 91%
rename from core/src/main/scala/spark/serializer/Serializer.scala
rename to core/src/main/scala/org/apache/spark/serializer/Serializer.scala
index dc94d42bb66ac23c843d2f04a649e2bb4593aec5..160cca4d6c54a69dca1232a38b5cbf94ad9e54d7 100644
--- a/core/src/main/scala/spark/serializer/Serializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
@@ -15,19 +15,19 @@
  * limitations under the License.
  */
 
-package spark.serializer
+package org.apache.spark.serializer
 
 import java.io.{EOFException, InputStream, OutputStream}
 import java.nio.ByteBuffer
 
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
 
-import spark.util.ByteBufferInputStream
+import org.apache.spark.util.{NextIterator, ByteBufferInputStream}
 
 
 /**
  * A serializer. Because some serialization libraries are not thread safe, this class is used to
- * create [[spark.serializer.SerializerInstance]] objects that do the actual serialization and are
+ * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual serialization and are
  * guaranteed to only be called from one thread at a time.
  */
 trait Serializer {
@@ -95,7 +95,7 @@ trait DeserializationStream {
    * Read the elements of this stream through an iterator. This can only be called once, as
    * reading each element will consume data from the input source.
    */
-  def asIterator: Iterator[Any] = new spark.util.NextIterator[Any] {
+  def asIterator: Iterator[Any] = new NextIterator[Any] {
     override protected def getNext() = {
       try {
         readObject[Any]()
diff --git a/core/src/main/scala/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
similarity index 98%
rename from core/src/main/scala/spark/serializer/SerializerManager.scala
rename to core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
index b7b24705a2a05d0822629b07be080cd3bd486e9e..2955986feced55d1a2f3ec00f174b5b793560f8c 100644
--- a/core/src/main/scala/spark/serializer/SerializerManager.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.serializer
+package org.apache.spark.serializer
 
 import java.util.concurrent.ConcurrentHashMap
 
diff --git a/core/src/main/scala/spark/storage/BlockException.scala b/core/src/main/scala/org/apache/spark/storage/BlockException.scala
similarity index 96%
rename from core/src/main/scala/spark/storage/BlockException.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockException.scala
index 8ebfaf3cbf6cdcb51a2b9916ab370f3930a45bb3..290dbce4f545a6d5ccf292529cec1f4eafc3a51e 100644
--- a/core/src/main/scala/spark/storage/BlockException.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockException.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 private[spark]
 case class BlockException(blockId: String, message: String) extends Exception(message)
diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.scala
similarity index 96%
rename from core/src/main/scala/spark/storage/BlockFetchTracker.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.scala
index 265e554ad88f485e79011366d452d40d533a4c26..2e0b0e6eda76525d557d8aeb6b5dc7ba9f0eeca5 100644
--- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 private[spark] trait BlockFetchTracker {
   def totalBlocks : Int
diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
similarity index 97%
rename from core/src/main/scala/spark/storage/BlockFetcherIterator.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
index 568783d893134069277ed428e2288b900aa9bbde..c91f0fc1ad8b3ab3c69c151c65a5479b9dac67f6 100644
--- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.nio.ByteBuffer
 import java.util.concurrent.LinkedBlockingQueue
@@ -26,13 +26,13 @@ import scala.collection.mutable.Queue
 
 import io.netty.buffer.ByteBuf
 
-import spark.Logging
-import spark.Utils
-import spark.SparkException
-import spark.network.BufferMessage
-import spark.network.ConnectionManagerId
-import spark.network.netty.ShuffleCopier
-import spark.serializer.Serializer
+import org.apache.spark.Logging
+import org.apache.spark.Utils
+import org.apache.spark.SparkException
+import org.apache.spark.network.BufferMessage
+import org.apache.spark.network.ConnectionManagerId
+import org.apache.spark.network.netty.ShuffleCopier
+import org.apache.spark.serializer.Serializer
 
 
 /**
diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
similarity index 99%
rename from core/src/main/scala/spark/storage/BlockManager.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 2a6ec2a55dddd893a2b14e2843fc1bf0e22dc63b..3299ac98d5511da4e3b894639295f448b43f2fca 100644
--- a/core/src/main/scala/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.io.{InputStream, OutputStream}
 import java.nio.{ByteBuffer, MappedByteBuffer}
@@ -29,11 +29,11 @@ import akka.util.duration._
 
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
 
-import spark.{Logging, SparkEnv, SparkException, Utils}
-import spark.io.CompressionCodec
-import spark.network._
-import spark.serializer.Serializer
-import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.{Logging, SparkEnv, SparkException, Utils}
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.network._
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap}
 
 import sun.nio.ch.DirectBuffer
 
diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
similarity index 94%
rename from core/src/main/scala/spark/storage/BlockManagerId.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
index b36a6176c0815bf6d990ef75bc1745484ff36a26..a22a80decc91744a8042ef931a0a6f648dd049c1 100644
--- a/core/src/main/scala/spark/storage/BlockManagerId.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}
 import java.util.concurrent.ConcurrentHashMap
-import spark.Utils
+import org.apache.spark.Utils
 
 /**
  * This class represent an unique identifier for a BlockManager.
@@ -92,13 +92,13 @@ private[spark] class BlockManagerId private (
 private[spark] object BlockManagerId {
 
   /**
-   * Returns a [[spark.storage.BlockManagerId]] for the given configuraiton.
+   * Returns a [[org.apache.spark.storage.BlockManagerId]] for the given configuraiton.
    *
    * @param execId ID of the executor.
    * @param host Host name of the block manager.
    * @param port Port of the block manager.
    * @param nettyPort Optional port for the Netty-based shuffle sender.
-   * @return A new [[spark.storage.BlockManagerId]].
+   * @return A new [[org.apache.spark.storage.BlockManagerId]].
    */
   def apply(execId: String, host: String, port: Int, nettyPort: Int) =
     getCachedBlockManagerId(new BlockManagerId(execId, host, port, nettyPort))
diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
similarity index 97%
rename from core/src/main/scala/spark/storage/BlockManagerMaster.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
index 76128e8cfffe9f0773f7807d950bc56554ebdc93..cf463d6ffc5cfc5500999fd9b905ec16df9fa4a4 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import akka.actor.ActorRef
 import akka.dispatch.{Await, Future}
 import akka.pattern.ask
 import akka.util.Duration
 
-import spark.{Logging, SparkException}
-import spark.storage.BlockManagerMessages._
+import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.storage.BlockManagerMessages._
 
 
 private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging {
diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
similarity index 98%
rename from core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
index b7a981d1014d48457fbe95d7981fa734d28c3691..baa4a1da50223b3700adf50b12494eceab25f275 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.util.{HashMap => JHashMap}
 
@@ -28,8 +28,8 @@ import akka.pattern.ask
 import akka.util.Duration
 import akka.util.duration._
 
-import spark.{Logging, Utils, SparkException}
-import spark.storage.BlockManagerMessages._
+import org.apache.spark.{Logging, Utils, SparkException}
+import org.apache.spark.storage.BlockManagerMessages._
 
 
 /**
diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
similarity index 99%
rename from core/src/main/scala/spark/storage/BlockManagerMessages.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
index 9375a9ca54825138d3988a6615af8d941ebff06a..24333a179c96467aa3f42e0aea0d69d0bb7edc6c 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.io.{Externalizable, ObjectInput, ObjectOutput}
 
diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala
similarity index 93%
rename from core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala
index 6e5fb43732905e66deedb0ea6e2bb17272e908a2..951503019f765001bd0313d17c663f6b3695afa2 100644
--- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import akka.actor.Actor
 
-import spark.storage.BlockManagerMessages._
+import org.apache.spark.storage.BlockManagerMessages._
 
 
 /**
diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala
similarity index 95%
rename from core/src/main/scala/spark/storage/BlockManagerSource.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala
index 2aecd1ea7167662c9ef858abb646a72ed856a9c3..24190cdd675e91c2a0d07d5a58d744f05ed6b86b 100644
--- a/core/src/main/scala/spark/storage/BlockManagerSource.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala
@@ -1,8 +1,8 @@
-package spark.storage
+package org.apache.spark.storage
 
 import com.codahale.metrics.{Gauge,MetricRegistry}
 
-import spark.metrics.source.Source
+import org.apache.spark.metrics.source.Source
 
 
 private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source {
diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
similarity index 97%
rename from core/src/main/scala/spark/storage/BlockManagerWorker.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
index 39064bce92c9476bf314301286ee6c6af53deb83..f4856020e55a63e58fe12c96b1d99b4f6f0afaf2 100644
--- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.nio.ByteBuffer
 
-import spark.{Logging, Utils}
-import spark.network._
+import org.apache.spark.{Logging, Utils}
+import org.apache.spark.network._
 
 /**
  * A network interface for BlockManager. Each slave should have one
diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala
similarity index 98%
rename from core/src/main/scala/spark/storage/BlockMessage.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockMessage.scala
index bcce26b7c14e94316b6818b9eef47156f0503cce..d8fa6a91d15f51d2e30e6a7f0c546c6e20970afa 100644
--- a/core/src/main/scala/spark/storage/BlockMessage.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.nio.ByteBuffer
 
 import scala.collection.mutable.StringBuilder
 import scala.collection.mutable.ArrayBuffer
 
-import spark.network._
+import org.apache.spark.network._
 
 private[spark] case class GetBlock(id: String)
 private[spark] case class GotBlock(id: String, data: ByteBuffer)
diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
similarity index 98%
rename from core/src/main/scala/spark/storage/BlockMessageArray.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
index ee2fc167d5b5198bfafebe634f6f3d1b8c2d8fc4..0aaf846b5b6606d3af2ba5fce6147f79050d5ea9 100644
--- a/core/src/main/scala/spark/storage/BlockMessageArray.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.nio.ByteBuffer
 
 import scala.collection.mutable.ArrayBuffer
 
-import spark._
-import spark.network._
+import org.apache.spark._
+import org.apache.spark.network._
 
 private[spark]
 class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging {
diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
similarity index 98%
rename from core/src/main/scala/spark/storage/BlockObjectWriter.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
index 3812009ca16709bda71e60366180cda77a2b2536..39f103297f90c86eb563da121c13797f9f76054b 100644
--- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 
 /**
diff --git a/core/src/main/scala/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala
similarity index 96%
rename from core/src/main/scala/spark/storage/BlockStore.scala
rename to core/src/main/scala/org/apache/spark/storage/BlockStore.scala
index c8db0022b0254919535bc610950162751678ab86..fa834371f4800e82fb1f6541d0db88db60b962a5 100644
--- a/core/src/main/scala/spark/storage/BlockStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.nio.ByteBuffer
 import scala.collection.mutable.ArrayBuffer
 
-import spark.Logging
+import org.apache.spark.Logging
 
 /**
  * Abstract class to store blocks
diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
similarity index 97%
rename from core/src/main/scala/spark/storage/DiskStore.scala
rename to core/src/main/scala/org/apache/spark/storage/DiskStore.scala
index b14497157e9e2123c14aaee58499bfb54bd290e5..fd945e065ccf2d84e17cce6298a55b184e90bf49 100644
--- a/core/src/main/scala/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile}
 import java.nio.ByteBuffer
@@ -28,12 +28,12 @@ import scala.collection.mutable.ArrayBuffer
 
 import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
 
-import spark.Utils
-import spark.executor.ExecutorExitCode
-import spark.serializer.{Serializer, SerializationStream}
-import spark.Logging
-import spark.network.netty.ShuffleSender
-import spark.network.netty.PathResolver
+import org.apache.spark.Utils
+import org.apache.spark.executor.ExecutorExitCode
+import org.apache.spark.serializer.{Serializer, SerializationStream}
+import org.apache.spark.Logging
+import org.apache.spark.network.netty.ShuffleSender
+import org.apache.spark.network.netty.PathResolver
 
 
 /**
diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
similarity index 99%
rename from core/src/main/scala/spark/storage/MemoryStore.scala
rename to core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
index 5a51f5cf313fa01a6de197a8c8337b4da72fba52..828dc0f22da7daf58a4d4cadc5f8da507c13ce9e 100644
--- a/core/src/main/scala/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.util.LinkedHashMap
 import java.util.concurrent.ArrayBlockingQueue
-import spark.{SizeEstimator, Utils}
+import org.apache.spark.{SizeEstimator, Utils}
 import java.nio.ByteBuffer
 import collection.mutable.ArrayBuffer
 
diff --git a/core/src/main/scala/spark/storage/PutResult.scala b/core/src/main/scala/org/apache/spark/storage/PutResult.scala
similarity index 97%
rename from core/src/main/scala/spark/storage/PutResult.scala
rename to core/src/main/scala/org/apache/spark/storage/PutResult.scala
index 3a0974fe15f13b1fdd1333bfdc86fea727d96c16..2eba2f06b5bfd96e02260525d6e3309dd84d2919 100644
--- a/core/src/main/scala/spark/storage/PutResult.scala
+++ b/core/src/main/scala/org/apache/spark/storage/PutResult.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.nio.ByteBuffer
 
diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
similarity index 96%
rename from core/src/main/scala/spark/storage/ShuffleBlockManager.scala
rename to core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
index 8a7a6f9ed3d0833aa08f5d346176512f7ca32124..9da11efb570d8d4c218c5884665a4ba0d237a07e 100644
--- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
-import spark.serializer.Serializer
+import org.apache.spark.serializer.Serializer
 
 
 private[spark]
diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
similarity index 97%
rename from core/src/main/scala/spark/storage/StorageLevel.scala
rename to core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
index f52650988c9dc9a4497486ca744922bba2948d88..755f1a760ee051dfd9c94ce6efb5d7765609060d 100644
--- a/core/src/main/scala/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}
 
@@ -23,7 +23,7 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}
  * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory,
  * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory
  * in a serialized format, and whether to replicate the RDD partitions on multiple nodes.
- * The [[spark.storage.StorageLevel$]] singleton object contains some static constants for
+ * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants for
  * commonly useful storage levels. To create your own storage level object, use the factor method
  * of the singleton object (`StorageLevel(...)`).
  */
diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
similarity index 98%
rename from core/src/main/scala/spark/storage/StorageUtils.scala
rename to core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
index 123b8f634571f9af7636ff8339881a9e57204a47..0bba1dac5409367051c5dea93a16d1f0f77ff7f6 100644
--- a/core/src/main/scala/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
-import spark.{Utils, SparkContext}
+import org.apache.spark.{Utils, SparkContext}
 import BlockManagerMasterActor.BlockStatus
 
 private[spark]
diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
similarity index 98%
rename from core/src/main/scala/spark/storage/ThreadingTest.scala
rename to core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
index b3ab1ff4b4dca7b324b7bf416ac993bc36c0c607..1d5afe9b0854e26aac306fd792912e4c95d9989e 100644
--- a/core/src/main/scala/spark/storage/ThreadingTest.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import akka.actor._
 
-import spark.KryoSerializer
+import org.apache.spark.KryoSerializer
 import java.util.concurrent.ArrayBlockingQueue
 import util.Random
 
diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
similarity index 98%
rename from core/src/main/scala/spark/ui/JettyUtils.scala
rename to core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index f66fe39905228e3167dabf33a1ab87740e7c01f5..cfa18f6ea4c101c470423f209d409bd7ad216640 100644
--- a/core/src/main/scala/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.ui
+package org.apache.spark.ui
 
 import javax.servlet.http.{HttpServletResponse, HttpServletRequest}
 
@@ -29,7 +29,7 @@ import org.eclipse.jetty.server.{Server, Request, Handler}
 import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler}
 import org.eclipse.jetty.util.thread.QueuedThreadPool
 
-import spark.Logging
+import org.apache.spark.Logging
 
 
 /** Utilities for launching a web server using Jetty's HTTP Server class */
diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/org/apache/spark/ui/Page.scala
similarity index 97%
rename from core/src/main/scala/spark/ui/Page.scala
rename to core/src/main/scala/org/apache/spark/ui/Page.scala
index 87376a19d86e9b2a1ca1cc97a464355a6ed50549..b2a069a37552d7d43acd4923220ca7feaf62c064 100644
--- a/core/src/main/scala/spark/ui/Page.scala
+++ b/core/src/main/scala/org/apache/spark/ui/Page.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.ui
+package org.apache.spark.ui
 
 private[spark] object Page extends Enumeration {
   val Stages, Storage, Environment, Executors = Value
diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
similarity index 88%
rename from core/src/main/scala/spark/ui/SparkUI.scala
rename to core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index 23ded44ba383de802aabbecaf136bfd53cbc09d4..4688effe0ade9793c53544a68018438ba2558385 100644
--- a/core/src/main/scala/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -15,18 +15,18 @@
  * limitations under the License.
  */
 
-package spark.ui
+package org.apache.spark.ui
 
 import javax.servlet.http.HttpServletRequest
 
 import org.eclipse.jetty.server.{Handler, Server}
 
-import spark.{Logging, SparkContext, SparkEnv, Utils}
-import spark.ui.env.EnvironmentUI
-import spark.ui.exec.ExecutorsUI
-import spark.ui.storage.BlockManagerUI
-import spark.ui.jobs.JobProgressUI
-import spark.ui.JettyUtils._
+import org.apache.spark.{Logging, SparkContext, SparkEnv, Utils}
+import org.apache.spark.ui.env.EnvironmentUI
+import org.apache.spark.ui.exec.ExecutorsUI
+import org.apache.spark.ui.storage.BlockManagerUI
+import org.apache.spark.ui.jobs.JobProgressUI
+import org.apache.spark.ui.JettyUtils._
 
 /** Top level user interface for Spark */
 private[spark] class SparkUI(sc: SparkContext) extends Logging {
@@ -83,5 +83,5 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging {
 
 private[spark] object SparkUI {
   val DEFAULT_PORT = "3030"
-  val STATIC_RESOURCE_DIR = "spark/ui/static"
+  val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static"
 }
diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
similarity index 98%
rename from core/src/main/scala/spark/ui/UIUtils.scala
rename to core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index 51bb18d888dc0fb4a8dfc216d7ed36d2a00feca2..ce1acf564cc9d2ed4ccfacd95c617295a472e1a9 100644
--- a/core/src/main/scala/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.ui
+package org.apache.spark.ui
 
 import scala.xml.Node
 
-import spark.SparkContext
+import org.apache.spark.SparkContext
 
 /** Utility functions for generating XML pages with spark content. */
 private[spark] object UIUtils {
diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
similarity index 95%
rename from core/src/main/scala/spark/ui/UIWorkloadGenerator.scala
rename to core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index 5ff0572f0a33b37e8c6d4a9579ef871bd923cabf..0ecb22d2f96f3f98f2adec3440bc1c9d90f7800c 100644
--- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.ui
+package org.apache.spark.ui
 
 import scala.util.Random
 
-import spark.SparkContext
-import spark.SparkContext._
-import spark.scheduler.cluster.SchedulingMode
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
+import org.apache.spark.scheduler.cluster.SchedulingMode
 
 
 /**
diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
similarity index 94%
rename from core/src/main/scala/spark/ui/env/EnvironmentUI.scala
rename to core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
index b1be1a27efe2fe66ff02db8a477153a5e5518a90..c5bf2acc9ec95d1ee11ed85d7ff2cff68f5ac405 100644
--- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.ui.env
+package org.apache.spark.ui.env
 
 import javax.servlet.http.HttpServletRequest
 
@@ -25,10 +25,10 @@ import scala.xml.Node
 
 import org.eclipse.jetty.server.Handler
 
-import spark.ui.JettyUtils._
-import spark.ui.UIUtils
-import spark.ui.Page.Environment
-import spark.SparkContext
+import org.apache.spark.ui.JettyUtils._
+import org.apache.spark.ui.UIUtils
+import org.apache.spark.ui.Page.Environment
+import org.apache.spark.SparkContext
 
 
 private[spark] class EnvironmentUI(sc: SparkContext) {
diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
similarity index 91%
rename from core/src/main/scala/spark/ui/exec/ExecutorsUI.scala
rename to core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
index 0a7021fbf866d74d30244386ac1191be2cb140fe..efe6b474e0082b6fc04ab59297a7aeacd4c97fdc 100644
--- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
@@ -1,4 +1,4 @@
-package spark.ui.exec
+package org.apache.spark.ui.exec
 
 import javax.servlet.http.HttpServletRequest
 
@@ -7,13 +7,13 @@ import scala.xml.Node
 
 import org.eclipse.jetty.server.Handler
 
-import spark.{ExceptionFailure, Logging, Utils, SparkContext}
-import spark.executor.TaskMetrics
-import spark.scheduler.cluster.TaskInfo
-import spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener}
-import spark.ui.JettyUtils._
-import spark.ui.Page.Executors
-import spark.ui.UIUtils
+import org.apache.spark.{ExceptionFailure, Logging, Utils, SparkContext}
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.scheduler.cluster.TaskInfo
+import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener}
+import org.apache.spark.ui.JettyUtils._
+import org.apache.spark.ui.Page.Executors
+import org.apache.spark.ui.UIUtils
 
 
 private[spark] class ExecutorsUI(val sc: SparkContext) {
diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
similarity index 95%
rename from core/src/main/scala/spark/ui/jobs/IndexPage.scala
rename to core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
index 8867a6c90c17d06e9cf24e252709754b672876f7..3b428effafad5c650708672cf9a5df23e3f3d1ed 100644
--- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.ui.jobs
+package org.apache.spark.ui.jobs
 
 import javax.servlet.http.HttpServletRequest
 
 import scala.xml.{NodeSeq, Node}
 
-import spark.scheduler.cluster.SchedulingMode
-import spark.ui.Page._
-import spark.ui.UIUtils._
+import org.apache.spark.scheduler.cluster.SchedulingMode
+import org.apache.spark.ui.Page._
+import org.apache.spark.ui.UIUtils._
 
 
 /** Page showing list of all ongoing and recently finished stages and pools*/
diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
similarity index 96%
rename from core/src/main/scala/spark/ui/jobs/JobProgressListener.scala
rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index 1d9767a83c8f65a75a7897eee406ae19a91b12f9..ae0222630007ffa3c9ea5a0fb7f9c4d2703b7bf8 100644
--- a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -1,12 +1,12 @@
-package spark.ui.jobs
+package org.apache.spark.ui.jobs
 
 import scala.Seq
 import scala.collection.mutable.{ListBuffer, HashMap, HashSet}
 
-import spark.{ExceptionFailure, SparkContext, Success, Utils}
-import spark.scheduler._
-import spark.scheduler.cluster.TaskInfo
-import spark.executor.TaskMetrics
+import org.apache.spark.{ExceptionFailure, SparkContext, Success, Utils}
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster.TaskInfo
+import org.apache.spark.executor.TaskMetrics
 import collection.mutable
 
 /**
diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
similarity index 86%
rename from core/src/main/scala/spark/ui/jobs/JobProgressUI.scala
rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
index c83f102ff32ff024dc1c776741209176bdeaf1f3..1bb7638bd9d88e25c02c45e43eedb42f3553f9ed 100644
--- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.ui.jobs
+package org.apache.spark.ui.jobs
 
 import akka.util.Duration
 
@@ -28,12 +28,12 @@ import org.eclipse.jetty.server.Handler
 import scala.Seq
 import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer}
 
-import spark.ui.JettyUtils._
-import spark.{ExceptionFailure, SparkContext, Success, Utils}
-import spark.scheduler._
+import org.apache.spark.ui.JettyUtils._
+import org.apache.spark.{ExceptionFailure, SparkContext, Success, Utils}
+import org.apache.spark.scheduler._
 import collection.mutable
-import spark.scheduler.cluster.SchedulingMode
-import spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.scheduler.cluster.SchedulingMode
+import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
 
 /** Web UI showing progress status of all jobs in the given SparkContext. */
 private[spark] class JobProgressUI(val sc: SparkContext) {
diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
similarity index 87%
rename from core/src/main/scala/spark/ui/jobs/PoolPage.scala
rename to core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
index 7fb74dce4065967e23ced4669e1eacb259cb923f..ce92b6932b3f0a5d25bca6d2f9763d35b1b816c2 100644
--- a/core/src/main/scala/spark/ui/jobs/PoolPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
@@ -1,13 +1,13 @@
-package spark.ui.jobs
+package org.apache.spark.ui.jobs
 
 import javax.servlet.http.HttpServletRequest
 
 import scala.xml.{NodeSeq, Node}
 import scala.collection.mutable.HashSet
 
-import spark.scheduler.Stage
-import spark.ui.UIUtils._
-import spark.ui.Page._
+import org.apache.spark.scheduler.Stage
+import org.apache.spark.ui.UIUtils._
+import org.apache.spark.ui.Page._
 
 /** Page showing specific pool details */
 private[spark] class PoolPage(parent: JobProgressUI) {
diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
similarity index 92%
rename from core/src/main/scala/spark/ui/jobs/PoolTable.scala
rename to core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
index 621828f9c3ea44899b0fee7b6bc6ceba9cc1837c..f31465e59daaaf9022141c3653013ffeb647870a 100644
--- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
@@ -1,11 +1,11 @@
-package spark.ui.jobs
+package org.apache.spark.ui.jobs
 
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
 import scala.xml.Node
 
-import spark.scheduler.Stage
-import spark.scheduler.cluster.Schedulable
+import org.apache.spark.scheduler.Stage
+import org.apache.spark.scheduler.cluster.Schedulable
 
 /** Table showing list of pools */
 private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressListener) {
diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
similarity index 95%
rename from core/src/main/scala/spark/ui/jobs/StagePage.scala
rename to core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index c2341475c7fffc776ec371d90f480ea8b8380067..2fe85bc0cf31255f16946e8c8001a2e859c08bd3 100644
--- a/core/src/main/scala/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.ui.jobs
+package org.apache.spark.ui.jobs
 
 import java.util.Date
 
@@ -23,12 +23,12 @@ import javax.servlet.http.HttpServletRequest
 
 import scala.xml.Node
 
-import spark.ui.UIUtils._
-import spark.ui.Page._
-import spark.util.Distribution
-import spark.{ExceptionFailure, Utils}
-import spark.scheduler.cluster.TaskInfo
-import spark.executor.TaskMetrics
+import org.apache.spark.ui.UIUtils._
+import org.apache.spark.ui.Page._
+import org.apache.spark.util.Distribution
+import org.apache.spark.{ExceptionFailure, Utils}
+import org.apache.spark.scheduler.cluster.TaskInfo
+import org.apache.spark.executor.TaskMetrics
 
 /** Page showing statistics and task list for a given stage */
 private[spark] class StagePage(parent: JobProgressUI) {
diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
similarity index 95%
rename from core/src/main/scala/spark/ui/jobs/StageTable.scala
rename to core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
index 2b1bc984fcab451d0d2561d889fe36473bd99d98..beb0574548603dfa506f897a0b27f3b16f4acb2b 100644
--- a/core/src/main/scala/spark/ui/jobs/StageTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
@@ -1,13 +1,13 @@
-package spark.ui.jobs
+package org.apache.spark.ui.jobs
 
 import java.util.Date
 
 import scala.xml.Node
 import scala.collection.mutable.HashSet
 
-import spark.Utils
-import spark.scheduler.cluster.{SchedulingMode, TaskInfo}
-import spark.scheduler.Stage
+import org.apache.spark.Utils
+import org.apache.spark.scheduler.cluster.{SchedulingMode, TaskInfo}
+import org.apache.spark.scheduler.Stage
 
 
 /** Page showing list of all ongoing and recently finished stages */
diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
similarity index 92%
rename from core/src/main/scala/spark/ui/storage/BlockManagerUI.scala
rename to core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
index 49ed069c75958f8eedee2236854f49b1b517836e..1d633d374a0a04abfc1e1172b342beee3debe77f 100644
--- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.ui.storage
+package org.apache.spark.ui.storage
 
 import akka.util.Duration
 
@@ -23,8 +23,8 @@ import javax.servlet.http.HttpServletRequest
 
 import org.eclipse.jetty.server.Handler
 
-import spark.{Logging, SparkContext}
-import spark.ui.JettyUtils._
+import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.ui.JettyUtils._
 
 /** Web UI showing storage status of all RDD's in the given SparkContext. */
 private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging {
diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala
similarity index 91%
rename from core/src/main/scala/spark/ui/storage/IndexPage.scala
rename to core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala
index fc6273c694542e8719d96fe213f4fa3875247b45..1eb4a7a85ea1ed52cc9e1d5cd3bcb0f85f403e66 100644
--- a/core/src/main/scala/spark/ui/storage/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package spark.ui.storage
+package org.apache.spark.ui.storage
 
 import javax.servlet.http.HttpServletRequest
 
 import scala.xml.Node
 
-import spark.storage.{RDDInfo, StorageUtils}
-import spark.Utils
-import spark.ui.UIUtils._
-import spark.ui.Page._
+import org.apache.spark.storage.{RDDInfo, StorageUtils}
+import org.apache.spark.Utils
+import org.apache.spark.ui.UIUtils._
+import org.apache.spark.ui.Page._
 
 /** Page showing list of RDD's currently stored in the cluster */
 private[spark] class IndexPage(parent: BlockManagerUI) {
diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
similarity index 94%
rename from core/src/main/scala/spark/ui/storage/RDDPage.scala
rename to core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
index b128a5614dcc7678fb3c8b17e6a82ed77b345083..37baf17f7a56468298e9631848c0b76391e106bb 100644
--- a/core/src/main/scala/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package spark.ui.storage
+package org.apache.spark.ui.storage
 
 import javax.servlet.http.HttpServletRequest
 
 import scala.xml.Node
 
-import spark.Utils
-import spark.storage.{StorageStatus, StorageUtils}
-import spark.storage.BlockManagerMasterActor.BlockStatus
-import spark.ui.UIUtils._
-import spark.ui.Page._
+import org.apache.spark.Utils
+import org.apache.spark.storage.{StorageStatus, StorageUtils}
+import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus
+import org.apache.spark.ui.UIUtils._
+import org.apache.spark.ui.Page._
 
 
 /** Page showing storage details for a given RDD */
diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
similarity index 99%
rename from core/src/main/scala/spark/util/AkkaUtils.scala
rename to core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
index 9233277bdb376c7df6ceca91855251a5cbcb2dba..d4c5065c3f5cfdeec6cba991d3919e719d84301c 100644
--- a/core/src/main/scala/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import akka.actor.{ActorSystem, ExtendedActorSystem}
 import com.typesafe.config.ConfigFactory
diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala
similarity index 98%
rename from core/src/main/scala/spark/util/BoundedPriorityQueue.scala
rename to core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala
index 0575497f5d8e87de35e345193cda23e158592263..0b51c23f7b2803498e527f56ad07e1e889aa1436 100644
--- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala
+++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import java.io.Serializable
 import java.util.{PriorityQueue => JPriorityQueue}
diff --git a/core/src/main/scala/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala
similarity index 96%
rename from core/src/main/scala/spark/util/ByteBufferInputStream.scala
rename to core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala
index 47a28e2f76e4067d095bbb38de34baa420cbc037..e214d2a519a20d94281af8e66f6c4d7716e7bbdf 100644
--- a/core/src/main/scala/spark/util/ByteBufferInputStream.scala
+++ b/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import java.io.InputStream
 import java.nio.ByteBuffer
-import spark.storage.BlockManager
+import org.apache.spark.storage.BlockManager
 
 /**
  * Reads data from a ByteBuffer, and optionally cleans it up using BlockManager.dispose()
diff --git a/core/src/main/scala/spark/util/Clock.scala b/core/src/main/scala/org/apache/spark/util/Clock.scala
similarity index 97%
rename from core/src/main/scala/spark/util/Clock.scala
rename to core/src/main/scala/org/apache/spark/util/Clock.scala
index aa71a5b4424640bff202c2efbc9d631bd0c9ee95..97c2b45aabf2875c2682dcd350b92c935376c099 100644
--- a/core/src/main/scala/spark/util/Clock.scala
+++ b/core/src/main/scala/org/apache/spark/util/Clock.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 /**
  * An interface to represent clocks, so that they can be mocked out in unit tests.
diff --git a/core/src/main/scala/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
similarity index 97%
rename from core/src/main/scala/spark/util/CompletionIterator.scala
rename to core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
index 210450892b00463ba05b455811464cd5a2a33026..dc15a38b29d702282986266b6674138b5c670c39 100644
--- a/core/src/main/scala/spark/util/CompletionIterator.scala
+++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 /**
  * Wrapper around an iterator which calls a completion method after it successfully iterates through all the elements
diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala
similarity index 98%
rename from core/src/main/scala/spark/util/Distribution.scala
rename to core/src/main/scala/org/apache/spark/util/Distribution.scala
index 5d4d7a6c50a5c68375343918fc86ce04e61dbdf8..33bf3562fe342b70424642f186514be16a7222fd 100644
--- a/core/src/main/scala/spark/util/Distribution.scala
+++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import java.io.PrintStream
 
diff --git a/core/src/main/scala/spark/util/IdGenerator.scala b/core/src/main/scala/org/apache/spark/util/IdGenerator.scala
similarity index 97%
rename from core/src/main/scala/spark/util/IdGenerator.scala
rename to core/src/main/scala/org/apache/spark/util/IdGenerator.scala
index 3422280559559a5e83c0b6954cdcb3bc63d527db..17e55f7996bf77574acf620ee2a9b6ef96b61ffc 100644
--- a/core/src/main/scala/spark/util/IdGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/util/IdGenerator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import java.util.concurrent.atomic.AtomicInteger
 
diff --git a/core/src/main/scala/spark/util/IntParam.scala b/core/src/main/scala/org/apache/spark/util/IntParam.scala
similarity index 97%
rename from core/src/main/scala/spark/util/IntParam.scala
rename to core/src/main/scala/org/apache/spark/util/IntParam.scala
index daf0d58fa25fb80fd62d83ed557f7b8261b5673a..626bb49eeae2fa5fca1471bbc2247540176d5b01 100644
--- a/core/src/main/scala/spark/util/IntParam.scala
+++ b/core/src/main/scala/org/apache/spark/util/IntParam.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 /**
  * An extractor object for parsing strings into integers.
diff --git a/core/src/main/scala/spark/util/MemoryParam.scala b/core/src/main/scala/org/apache/spark/util/MemoryParam.scala
similarity index 95%
rename from core/src/main/scala/spark/util/MemoryParam.scala
rename to core/src/main/scala/org/apache/spark/util/MemoryParam.scala
index 298562323ae05124493d8911056e49c1e3ad3c1d..0ee6707826ddb82ac0665caaacc3e19ea034564c 100644
--- a/core/src/main/scala/spark/util/MemoryParam.scala
+++ b/core/src/main/scala/org/apache/spark/util/MemoryParam.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
-import spark.Utils
+import org.apache.spark.Utils
 
 /**
  * An extractor object for parsing JVM memory strings, such as "10g", into an Int representing
diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
similarity index 97%
rename from core/src/main/scala/spark/util/MetadataCleaner.scala
rename to core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
index 92909e09590a362bbef6c3cf6ad698697fe1cc12..a430a75451173b197378981c8836bb8904f340df 100644
--- a/core/src/main/scala/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors}
 import java.util.{TimerTask, Timer}
-import spark.Logging
+import org.apache.spark.Logging
 
 
 /**
diff --git a/core/src/main/scala/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala
similarity index 97%
rename from core/src/main/scala/spark/util/MutablePair.scala
rename to core/src/main/scala/org/apache/spark/util/MutablePair.scala
index 78d404e66b1e205e92064183a347f46d1eef2f03..34f1f6606fc3fbbdfca834c7843a6d9f2fe4f7c8 100644
--- a/core/src/main/scala/spark/util/MutablePair.scala
+++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 
 /**
diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/org/apache/spark/util/NextIterator.scala
similarity index 98%
rename from core/src/main/scala/spark/util/NextIterator.scala
rename to core/src/main/scala/org/apache/spark/util/NextIterator.scala
index 22163ece8dd21bee8c13418bb7369027a3c2a49c..8266e5e495efcc88d58b0e7f63948fb34f5211de 100644
--- a/core/src/main/scala/spark/util/NextIterator.scala
+++ b/core/src/main/scala/org/apache/spark/util/NextIterator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 /** Provides a basic/boilerplate Iterator implementation. */
 private[spark] abstract class NextIterator[U] extends Iterator[U] {
diff --git a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala b/core/src/main/scala/org/apache/spark/util/RateLimitedOutputStream.scala
similarity index 98%
rename from core/src/main/scala/spark/util/RateLimitedOutputStream.scala
rename to core/src/main/scala/org/apache/spark/util/RateLimitedOutputStream.scala
index 00f782bbe7620f941224e320175c043ecb609cd2..47e1b450043b32b8d9178dc6dce40a9768c318d7 100644
--- a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala
+++ b/core/src/main/scala/org/apache/spark/util/RateLimitedOutputStream.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import scala.annotation.tailrec
 
diff --git a/core/src/main/scala/spark/util/SerializableBuffer.scala b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala
similarity index 98%
rename from core/src/main/scala/spark/util/SerializableBuffer.scala
rename to core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala
index 7e6842628a6379f2b4969006ea4e0074c11e0ba7..f2b1ad7d0e91d667cd01b7afc16e91eb78d3fcea 100644
--- a/core/src/main/scala/spark/util/SerializableBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import java.nio.ByteBuffer
 import java.io.{IOException, ObjectOutputStream, EOFException, ObjectInputStream}
diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala
similarity index 99%
rename from core/src/main/scala/spark/util/StatCounter.scala
rename to core/src/main/scala/org/apache/spark/util/StatCounter.scala
index 76358d4151bc9bd53e7cd0892a678087691b0462..020d5edba9530d4eefee2e80579b795b0af73860 100644
--- a/core/src/main/scala/spark/util/StatCounter.scala
+++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 /**
  * A class for tracking the statistics of a set of numbers (count, mean and variance) in a
diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala
similarity index 95%
rename from core/src/main/scala/spark/util/TimeStampedHashMap.scala
rename to core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala
index 07772a0afbefe3e95ff0ddd1b50c6bad27fb1b99..277de2f8a6caa941ca15c4ce7d3d67b663bd7ed8 100644
--- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala
@@ -15,13 +15,14 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import java.util.concurrent.ConcurrentHashMap
 import scala.collection.JavaConversions
 import scala.collection.mutable.Map
 import scala.collection.immutable
-import spark.scheduler.MapStatus
+import org.apache.spark.scheduler.MapStatus
+import org.apache.spark.Logging
 
 /**
  * This is a custom implementation of scala.collection.mutable.Map which stores the insertion
@@ -29,7 +30,7 @@ import spark.scheduler.MapStatus
  * threshold time can them be removed using the clearOldValues method. This is intended to be a drop-in
  * replacement of scala.collection.mutable.HashMap.
  */
-class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging {
+class TimeStampedHashMap[A, B] extends Map[A, B]() with Logging {
   val internalMap = new ConcurrentHashMap[A, (B, Long)]()
 
   def get(key: A): Option[B] = {
diff --git a/core/src/main/scala/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala
similarity index 98%
rename from core/src/main/scala/spark/util/TimeStampedHashSet.scala
rename to core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala
index 41e3fd8cba1cf4bb67611da21e5c3baffaabeee1..26983138ff0da984c3e97b4149ec4ebd16b712e8 100644
--- a/core/src/main/scala/spark/util/TimeStampedHashSet.scala
+++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import scala.collection.mutable.Set
 import scala.collection.JavaConversions
diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala
similarity index 97%
rename from core/src/main/scala/spark/util/Vector.scala
rename to core/src/main/scala/org/apache/spark/util/Vector.scala
index a47cac3b96830301c62f1fc0cdba19204bc3f4b7..fe710c58acc44b53a0b962033fe3f8ad6435572c 100644
--- a/core/src/main/scala/spark/util/Vector.scala
+++ b/core/src/main/scala/org/apache/spark/util/Vector.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 class Vector(val elements: Array[Double]) extends Serializable {
   def length = elements.length
@@ -130,7 +130,7 @@ object Vector {
 
   implicit def doubleToMultiplier(num: Double) = new Multiplier(num)
 
-  implicit object VectorAccumParam extends spark.AccumulatorParam[Vector] {
+  implicit object VectorAccumParam extends org.apache.spark.AccumulatorParam[Vector] {
     def addInPlace(t1: Vector, t2: Vector) = t1 + t2
 
     def zero(initialValue: Vector) = Vector.zeros(initialValue.length)
diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties
index 2b31ddf2ebca651d7e2273c013e1e2dd9345b056..056a15845698c99acfdcc707cf8fe50ff3af51dd 100644
--- a/core/src/test/resources/test_metrics_config.properties
+++ b/core/src/test/resources/test_metrics_config.properties
@@ -1,6 +1,6 @@
 *.sink.console.period = 10
 *.sink.console.unit = seconds
-*.source.jvm.class = spark.metrics.source.JvmSource
+*.source.jvm.class = org.apache.spark.metrics.source.JvmSource
 master.sink.console.period = 20
 master.sink.console.unit = minutes
 
diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties
index d5479f02980af9c96bc3202e109277a435ea69d4..6f5ecea93a395af5140a868befd56ff267b667a5 100644
--- a/core/src/test/resources/test_metrics_system.properties
+++ b/core/src/test/resources/test_metrics_system.properties
@@ -1,7 +1,7 @@
 *.sink.console.period = 10
 *.sink.console.unit = seconds
-test.sink.console.class = spark.metrics.sink.ConsoleSink
-test.sink.dummy.class = spark.metrics.sink.DummySink
-test.source.dummy.class = spark.metrics.source.DummySource
+test.sink.console.class = org.apache.spark.metrics.sink.ConsoleSink
+test.sink.dummy.class = org.apache.spark.metrics.sink.DummySink
+test.source.dummy.class = org.apache.spark.metrics.source.DummySource
 test.sink.console.period = 20
 test.sink.console.unit = minutes
diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/AccumulatorSuite.scala
rename to core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
index 0af175f3168d5d7b7f7b159f45d10f9179719747..4434f3b87c920aa9b9142b3fa1cb77753d63b2da 100644
--- a/core/src/test/scala/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 import org.scalatest.matchers.ShouldMatchers
@@ -23,7 +23,7 @@ import collection.mutable
 import java.util.Random
 import scala.math.exp
 import scala.math.signum
-import spark.SparkContext._
+import org.apache.spark.SparkContext._
 
 class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
 
diff --git a/core/src/test/scala/spark/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/BroadcastSuite.scala
rename to core/src/test/scala/org/apache/spark/BroadcastSuite.scala
index 785721ece86ac2b4b557a97fbb5b254075e7d0d4..b3a53d928b46e2b9dc6802c3eb78e70e4f8514c8 100644
--- a/core/src/test/scala/spark/BroadcastSuite.scala
+++ b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 
diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/CheckpointSuite.scala
rename to core/src/test/scala/org/apache/spark/CheckpointSuite.scala
index 966dede2be12a8ba626f49c69320e76057bb59d6..23b14f4245e9217886da80643f9f1f2b3217b850 100644
--- a/core/src/test/scala/spark/CheckpointSuite.scala
+++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 import java.io.File
-import spark.rdd._
-import spark.SparkContext._
+import org.apache.spark.rdd._
+import org.apache.spark.SparkContext._
 import storage.StorageLevel
 
 class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
diff --git a/core/src/test/scala/spark/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/ClosureCleanerSuite.scala
rename to core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala
index 7d2831e19c9e75c31063cc9533fd25f21ebf58c5..8494899b984936311babcd495eb14b3722273f28 100644
--- a/core/src/test/scala/spark/ClosureCleanerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io.NotSerializableException
 
 import org.scalatest.FunSuite
-import spark.LocalSparkContext._
+import org.apache.spark.LocalSparkContext._
 import SparkContext._
 
 class ClosureCleanerSuite extends FunSuite {
diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/DistributedSuite.scala
rename to core/src/test/scala/org/apache/spark/DistributedSuite.scala
index e11efe459c8adb74ffba3ee84c45f57529f44220..7a856d40817ee608151079a3224e4867c4285b70 100644
--- a/core/src/test/scala/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import network.ConnectionManagerId
 import org.scalatest.FunSuite
diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
similarity index 93%
rename from core/src/test/scala/spark/DriverSuite.scala
rename to core/src/test/scala/org/apache/spark/DriverSuite.scala
index 553c0309f66fa5d4c5ca58202e3f4723d5fdd018..b08aad1a6fd8fd865ac0ded82a97348a1430ce9e 100644
--- a/core/src/test/scala/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io.File
 
@@ -34,7 +34,7 @@ class DriverSuite extends FunSuite with Timeouts {
     val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]"))
     forAll(masters) { (master: String) =>
       failAfter(30 seconds) {
-        Utils.execute(Seq("./spark-class", "spark.DriverWithoutCleanup", master),
+        Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master),
           new File(System.getenv("SPARK_HOME")))
       }
     }
diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/FailureSuite.scala
rename to core/src/test/scala/org/apache/spark/FailureSuite.scala
index 5b133cdd6e867947bb324b068fe352cefd3c1178..ee89a7a38721e3aab01157f91d66ca8613b5b9a1 100644
--- a/core/src/test/scala/spark/FailureSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 
diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/FileServerSuite.scala
rename to core/src/test/scala/org/apache/spark/FileServerSuite.scala
index 242ae971f859b5015252e16f6fb9109559c58c06..35d1d41af175b419f1fa18cbff60cdc5bf98eeff 100644
--- a/core/src/test/scala/spark/FileServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import com.google.common.io.Files
 import org.scalatest.FunSuite
diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/FileSuite.scala
rename to core/src/test/scala/org/apache/spark/FileSuite.scala
index 1e2c257c4b224e6c8bd89f0b894433b3b171d423..7b82a4cdd9cfd2b5f5b06239d5fc96d15bcade08 100644
--- a/core/src/test/scala/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.io.{FileWriter, PrintWriter, File}
 
diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
similarity index 98%
rename from core/src/test/scala/spark/JavaAPISuite.java
rename to core/src/test/scala/org/apache/spark/JavaAPISuite.java
index c337c49268ad759deaee29bf52f2fdbadd727e93..8a869c9005fb61280adfec2550bc3f12253e12b4 100644
--- a/core/src/test/scala/spark/JavaAPISuite.java
+++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark;
+package org.apache.spark;
 
 import java.io.File;
 import java.io.IOException;
@@ -38,15 +38,15 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import spark.api.java.JavaDoubleRDD;
-import spark.api.java.JavaPairRDD;
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.*;
-import spark.partial.BoundedDouble;
-import spark.partial.PartialResult;
-import spark.storage.StorageLevel;
-import spark.util.StatCounter;
+import org.apache.spark.api.java.JavaDoubleRDD;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.*;
+import org.apache.spark.partial.BoundedDouble;
+import org.apache.spark.partial.PartialResult;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.util.StatCounter;
 
 
 // The test suite itself is Serializable so that anonymous Function implementations can be
diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/KryoSerializerSuite.scala
rename to core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala
index 7568a0bf6591cf4dd8ab5d42e0d32c119b3ef778..d7b23c93fecea133ecb75a3a93e0def1de847fe8 100644
--- a/core/src/test/scala/spark/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import scala.collection.mutable
 
@@ -167,7 +167,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
   }
 
   override def beforeAll() {
-    System.setProperty("spark.serializer", "spark.KryoSerializer")
+    System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer")
     System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName)
     super.beforeAll()
   }
diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
similarity index 98%
rename from core/src/test/scala/spark/LocalSparkContext.scala
rename to core/src/test/scala/org/apache/spark/LocalSparkContext.scala
index ddc212d290dcca421c27bbac0f19ce51e64ab7fa..6ec124da9c7b17a933717e0f195119fc558b9f5f 100644
--- a/core/src/test/scala/spark/LocalSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.Suite
 import org.scalatest.BeforeAndAfterEach
diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
similarity index 97%
rename from core/src/test/scala/spark/MapOutputTrackerSuite.scala
rename to core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index c21f3331d0876b769092e3454f496b96b7cda49a..6013320eaab730c202acce87de87714c99af65fb 100644
--- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 
 import akka.actor._
-import spark.scheduler.MapStatus
-import spark.storage.BlockManagerId
-import spark.util.AkkaUtils
+import org.apache.spark.scheduler.MapStatus
+import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.util.AkkaUtils
 
 class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
 
diff --git a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/PairRDDFunctionsSuite.scala
rename to core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala
index 328b3b5497ea884f4c9039c672f24398b4ca2fd5..f79752b34efb86487161abfb11c3572a9f3661be 100644
--- a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashSet
@@ -23,7 +23,7 @@ import scala.collection.mutable.HashSet
 import org.scalatest.FunSuite
 
 import com.google.common.io.Files
-import spark.SparkContext._
+import org.apache.spark.SparkContext._
 
 
 class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext {
diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala
similarity index 86%
rename from core/src/test/scala/spark/PartitionPruningRDDSuite.scala
rename to core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala
index 88352b639f9b1d953246bc438a82a0e364a7f1bf..adbe805916b08ed8141892997497855d4154d6db 100644
--- a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala
@@ -1,8 +1,8 @@
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
-import spark.SparkContext._
-import spark.rdd.PartitionPruningRDD
+import org.apache.spark.SparkContext._
+import org.apache.spark.rdd.PartitionPruningRDD
 
 
 class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext {
diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/PartitioningSuite.scala
rename to core/src/test/scala/org/apache/spark/PartitioningSuite.scala
index b1e0b2b4d061cb3af2265d6acc081f03ed8a80cb..7669cf6fb147a84ef85d56a9938f8b80f1a65119 100644
--- a/core/src/test/scala/spark/PartitioningSuite.scala
+++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 import scala.collection.mutable.ArrayBuffer
 import SparkContext._
-import spark.util.StatCounter
+import org.apache.spark.util.StatCounter
 import scala.math.abs
 
 class PartitioningSuite extends FunSuite with SharedSparkContext {
diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/PipedRDDSuite.scala
rename to core/src/test/scala/org/apache/spark/PipedRDDSuite.scala
index 35c04710a39556aad7057dc507c24618fa8129b3..2e851d892dd0d5f4ec3628a2d945c172d3b16d0b 100644
--- a/core/src/test/scala/spark/PipedRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 import SparkContext._
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/org/apache/spark/RDDSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/RDDSuite.scala
rename to core/src/test/scala/org/apache/spark/RDDSuite.scala
index e306952bbd63fdc53c54f38db74f278d2080b0ea..342ba8adb20b742eb96e4ffb2defe0db64097b96 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/RDDSuite.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import scala.collection.mutable.HashMap
 import org.scalatest.FunSuite
 import org.scalatest.concurrent.Timeouts._
 import org.scalatest.time.{Span, Millis}
-import spark.SparkContext._
-import spark.rdd._
+import org.apache.spark.SparkContext._
+import org.apache.spark.rdd._
 import scala.collection.parallel.mutable
 
 class RDDSuite extends FunSuite with SharedSparkContext {
diff --git a/core/src/test/scala/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
similarity index 98%
rename from core/src/test/scala/spark/SharedSparkContext.scala
rename to core/src/test/scala/org/apache/spark/SharedSparkContext.scala
index 70c24515be1571defaefc6c6605c9bad7b77ae02..97cbca09bfa2687939894b8cf5281b4cbbfa6c95 100644
--- a/core/src/test/scala/spark/SharedSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.Suite
 import org.scalatest.BeforeAndAfterAll
diff --git a/core/src/test/scala/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala
similarity index 98%
rename from core/src/test/scala/spark/ShuffleNettySuite.scala
rename to core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala
index 6bad6c1d13deaffdfecea2c2e2179b95ad9c8926..e121b162ad9e6243c437a32473603e496e9efd9e 100644
--- a/core/src/test/scala/spark/ShuffleNettySuite.scala
+++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.BeforeAndAfterAll
 
diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
similarity index 95%
rename from core/src/test/scala/spark/ShuffleSuite.scala
rename to core/src/test/scala/org/apache/spark/ShuffleSuite.scala
index 8745689c70bca55f8b11a6579cfdbc0ac9e00e2a..357175e89ee719e5c8729856999386ea5db376c2 100644
--- a/core/src/test/scala/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 import org.scalatest.matchers.ShouldMatchers
 
-import spark.SparkContext._
-import spark.ShuffleSuite.NonJavaSerializableClass
-import spark.rdd.{SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD}
-import spark.util.MutablePair
+import org.apache.spark.SparkContext._
+import org.apache.spark.ShuffleSuite.NonJavaSerializableClass
+import org.apache.spark.rdd.{SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD}
+import org.apache.spark.util.MutablePair
 
 
 class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
@@ -54,7 +54,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
     // If the Kryo serializer is not used correctly, the shuffle would fail because the
     // default Java serializer cannot handle the non serializable class.
     val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)](
-      b, new HashPartitioner(NUM_BLOCKS)).setSerializer(classOf[spark.KryoSerializer].getName)
+      b, new HashPartitioner(NUM_BLOCKS)).setSerializer(classOf[KryoSerializer].getName)
     val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId
 
     assert(c.count === 10)
@@ -76,7 +76,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
     // If the Kryo serializer is not used correctly, the shuffle would fail because the
     // default Java serializer cannot handle the non serializable class.
     val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)](
-      b, new HashPartitioner(3)).setSerializer(classOf[spark.KryoSerializer].getName)
+      b, new HashPartitioner(3)).setSerializer(classOf[KryoSerializer].getName)
     assert(c.count === 10)
   }
 
@@ -92,7 +92,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
     // NOTE: The default Java serializer doesn't create zero-sized blocks.
     //       So, use Kryo
     val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10))
-      .setSerializer(classOf[spark.KryoSerializer].getName)
+      .setSerializer(classOf[KryoSerializer].getName)
 
     val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId
     assert(c.count === 4)
diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/SizeEstimatorSuite.scala
rename to core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala
index 1ef812dfbddae6aeb56f18b2ee02487c74ca744f..214ac74898f8b3d795c25f25a2382ff2777fef84 100644
--- a/core/src/test/scala/spark/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfterAll
diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/org/apache/spark/SortingSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/SortingSuite.scala
rename to core/src/test/scala/org/apache/spark/SortingSuite.scala
index b933c4aab835ce6a971c00b6564b99934c366bad..f4fa9511dda9ab93799024cb8f996bed713844e8 100644
--- a/core/src/test/scala/spark/SortingSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SortingSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
diff --git a/core/src/test/scala/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
similarity index 96%
rename from core/src/test/scala/spark/SparkContextInfoSuite.scala
rename to core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
index 6d50bf5e1b31b10718edcbc5f50394a501135fe7..939fe518012d0c17c9b319c2d94c08f9ffa7e7f6 100644
--- a/core/src/test/scala/spark/SparkContextInfoSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
-import spark.SparkContext._
+import org.apache.spark.SparkContext._
 
 class SparkContextInfoSuite extends FunSuite with LocalSparkContext {
   test("getPersistentRDDs only returns RDDs that are marked as cached") {
@@ -57,4 +57,4 @@ class SparkContextInfoSuite extends FunSuite with LocalSparkContext {
     rdd.collect()
     assert(sc.getRDDStorageInfo.size === 1)
   }
-}
\ No newline at end of file
+}
diff --git a/core/src/test/scala/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/ThreadingSuite.scala
rename to core/src/test/scala/org/apache/spark/ThreadingSuite.scala
index f2acd0bd3c1db30b2dd56de2b7b5ad9dea9b97f0..69383ddfb8bc7471653ad5d9234fa45f9bbb9737 100644
--- a/core/src/test/scala/spark/ThreadingSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import java.util.concurrent.Semaphore
 import java.util.concurrent.atomic.AtomicBoolean
diff --git a/core/src/test/scala/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
similarity index 96%
rename from core/src/test/scala/spark/UnpersistSuite.scala
rename to core/src/test/scala/org/apache/spark/UnpersistSuite.scala
index 93977d16f4ccd1361eda641e5d356846c07ea7f2..46a2da172407f009f4895e5d4d7efa362b0bf398 100644
--- a/core/src/test/scala/spark/UnpersistSuite.scala
+++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.FunSuite
 import org.scalatest.concurrent.Timeouts._
 import org.scalatest.time.{Span, Millis}
-import spark.SparkContext._
+import org.apache.spark.SparkContext._
 
 class UnpersistSuite extends FunSuite with LocalSparkContext {
   test("unpersist RDD") {
diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/UtilsSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/UtilsSuite.scala
rename to core/src/test/scala/org/apache/spark/UtilsSuite.scala
index 98a6c1a1c9e336189fdca53c27cb59b5b6cb52d6..3a908720a85b85b39966745b03d2c3ddbb235468 100644
--- a/core/src/test/scala/spark/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/UtilsSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import com.google.common.base.Charsets
 import com.google.common.io.Files
diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/ZippedPartitionsSuite.scala
rename to core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala
index bb5d379273b3db9ffa2f7aa0ae3d1f093daff7d3..618b9c113b849d765ee1fad04ec4ac415b6c8a96 100644
--- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import scala.collection.immutable.NumericRange
 
diff --git a/core/src/test/scala/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/io/CompressionCodecSuite.scala
rename to core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
index 1ba82fe2b92b0b53d27c4ab0a7d1cc9cf1b6b828..fd6f69041a94fc3cb0d2108869c62785bd12a397 100644
--- a/core/src/test/scala/spark/io/CompressionCodecSuite.scala
+++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.io
+package org.apache.spark.io
 
 import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
 
diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala
similarity index 82%
rename from core/src/test/scala/spark/metrics/MetricsConfigSuite.scala
rename to core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala
index b0213b62d9c993168a4e3385d801134e239df4e7..58c94a162d4a4122fe0dcef9a0defedc773c5cda 100644
--- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.metrics
+package org.apache.spark.metrics
 
 import org.scalatest.{BeforeAndAfter, FunSuite}
 
@@ -35,7 +35,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter {
 
     val property = conf.getInstance("random")
     assert(property.size() === 3)
-    assert(property.getProperty("sink.servlet.class") === "spark.metrics.sink.MetricsServlet")
+    assert(property.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet")
     assert(property.getProperty("sink.servlet.uri") === "/metrics/json")
     assert(property.getProperty("sink.servlet.sample") === "false")
   }
@@ -48,8 +48,8 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter {
     assert(masterProp.size() === 6)
     assert(masterProp.getProperty("sink.console.period") === "20")
     assert(masterProp.getProperty("sink.console.unit") === "minutes")
-    assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource")
-    assert(masterProp.getProperty("sink.servlet.class") === "spark.metrics.sink.MetricsServlet")
+    assert(masterProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource")
+    assert(masterProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet")
     assert(masterProp.getProperty("sink.servlet.uri") === "/metrics/master/json")
     assert(masterProp.getProperty("sink.servlet.sample") === "false")
 
@@ -57,8 +57,8 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter {
     assert(workerProp.size() === 6)
     assert(workerProp.getProperty("sink.console.period") === "10")
     assert(workerProp.getProperty("sink.console.unit") === "seconds")
-    assert(workerProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource")
-    assert(workerProp.getProperty("sink.servlet.class") === "spark.metrics.sink.MetricsServlet")
+    assert(workerProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource")
+    assert(workerProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet")
     assert(workerProp.getProperty("sink.servlet.uri") === "/metrics/json")
     assert(workerProp.getProperty("sink.servlet.sample") === "false")
   }
@@ -73,7 +73,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter {
     val masterProp = conf.getInstance("master")
     val sourceProps = conf.subProperties(masterProp, MetricsSystem.SOURCE_REGEX)
     assert(sourceProps.size === 1)
-    assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource")
+    assert(sourceProps("jvm").getProperty("class") === "org.apache.spark.metrics.source.JvmSource")
 
     val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX)
     assert(sinkProps.size === 2)
diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
similarity index 93%
rename from core/src/test/scala/spark/metrics/MetricsSystemSuite.scala
rename to core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
index dc65ac6994b74e8d5e7290ee34860d6667512a07..7181333adfe689461d7ac93d131c8dd5dd4f4872 100644
--- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
@@ -15,9 +15,10 @@
  * limitations under the License.
  */
 
-package spark.metrics
+package org.apache.spark.metrics
 
 import org.scalatest.{BeforeAndAfter, FunSuite}
+import org.apache.spark.deploy.master.MasterSource
 
 class MetricsSystemSuite extends FunSuite with BeforeAndAfter {
   var filePath: String = _
@@ -46,7 +47,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter {
     assert(sinks.length === 1)
     assert(!metricsSystem.getServletHandlers.isEmpty)
 
-    val source = new spark.deploy.master.MasterSource(null)
+    val source = new MasterSource(null)
     metricsSystem.registerSource(source)
     assert(sources.length === 1)
   }
diff --git a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
similarity index 95%
rename from core/src/test/scala/spark/rdd/JdbcRDDSuite.scala
rename to core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
index dc8ca941c105a3c43e59888cff7b0ae98a64a300..3d39a31252e5e7a4cd5c09657a692d7b58dcc01c 100644
--- a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark
+package org.apache.spark
 
 import org.scalatest.{ BeforeAndAfter, FunSuite }
-import spark.SparkContext._
-import spark.rdd.JdbcRDD
+import org.apache.spark.SparkContext._
+import org.apache.spark.rdd.JdbcRDD
 import java.sql._
 
 class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
diff --git a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala
similarity index 99%
rename from core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
rename to core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala
index d1276d541f7336fb530d7736bfccddf7ef417b2b..a80afdee7e76943b2e6a2591aa8d38fe86543337 100644
--- a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.rdd
+package org.apache.spark.rdd
 
 import scala.collection.immutable.NumericRange
 
diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
similarity index 96%
rename from core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
rename to core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 3b4a0d52fc79e98ba4bf0d8e28078dae53cf07d1..94df282b2849f25a4a7be5cd36334123e9e2811f 100644
--- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -15,26 +15,26 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import scala.collection.mutable.{Map, HashMap}
 
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
 
-import spark.LocalSparkContext
-import spark.MapOutputTracker
-import spark.RDD
-import spark.SparkContext
-import spark.Partition
-import spark.TaskContext
-import spark.{Dependency, ShuffleDependency, OneToOneDependency}
-import spark.{FetchFailed, Success, TaskEndReason}
-import spark.storage.{BlockManagerId, BlockManagerMaster}
-
-import spark.scheduler.cluster.Pool
-import spark.scheduler.cluster.SchedulingMode
-import spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.LocalSparkContext
+import org.apache.spark.MapOutputTracker
+import org.apache.spark.RDD
+import org.apache.spark.SparkContext
+import org.apache.spark.Partition
+import org.apache.spark.TaskContext
+import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency}
+import org.apache.spark.{FetchFailed, Success, TaskEndReason}
+import org.apache.spark.storage.{BlockManagerId, BlockManagerMaster}
+
+import org.apache.spark.scheduler.cluster.Pool
+import org.apache.spark.scheduler.cluster.SchedulingMode
+import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
 
 /**
  * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler
diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
rename to core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
index bb9e715f95e217bd2c2b7c5f7a3bcbe90c782ae9..f5b3e972227e23b45a48b42f32afb263327fae7e 100644
--- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import java.util.Properties
 import java.util.concurrent.LinkedBlockingQueue
 import org.scalatest.FunSuite
 import org.scalatest.matchers.ShouldMatchers
 import scala.collection.mutable
-import spark._
-import spark.SparkContext._
+import org.apache.spark._
+import org.apache.spark.SparkContext._
 
 
 class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers {
diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
similarity index 96%
rename from core/src/test/scala/spark/scheduler/SparkListenerSuite.scala
rename to core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index 392d67d67bfdf65baeb3b03f4dd4b1b3e98fc625..aac7c207cbe0c8a6c3f3be75649815c9f57d058a 100644
--- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import org.scalatest.FunSuite
-import spark.{SparkContext, LocalSparkContext}
+import org.apache.spark.{SparkContext, LocalSparkContext}
 import scala.collection.mutable
 import org.scalatest.matchers.ShouldMatchers
-import spark.SparkContext._
+import org.apache.spark.SparkContext._
 
 /**
  *
diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
similarity index 88%
rename from core/src/test/scala/spark/scheduler/TaskContextSuite.scala
rename to core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
index 95a6eee2fcf2137eea38a083c842814681351a6a..0347cc02d7a617a8ddded3e9923c8b29e417736a 100644
--- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.scheduler
+package org.apache.spark.scheduler
 
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
-import spark.TaskContext
-import spark.RDD
-import spark.SparkContext
-import spark.Partition
-import spark.LocalSparkContext
+import org.apache.spark.TaskContext
+import org.apache.spark.RDD
+import org.apache.spark.SparkContext
+import org.apache.spark.Partition
+import org.apache.spark.LocalSparkContext
 
 class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
 
diff --git a/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala
rename to core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
index abfdabf5fe634f5c135a7f1c04467120a4c8cc4c..92ad9f09b2d89b2d212d2944e1905d12b89f30bd 100644
--- a/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
 
-import spark._
-import spark.scheduler._
-import spark.scheduler.cluster._
+import org.apache.spark._
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster._
 import scala.collection.mutable.ArrayBuffer
 
 import java.util.Properties
diff --git a/core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
rename to core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
index 5a0b949ef5f65e87595b294d18df0e7e14e19ebd..a4f63baf3d85759934628444e7bc686b97077341 100644
--- a/core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
@@ -15,18 +15,18 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable
 
 import org.scalatest.FunSuite
 
-import spark._
-import spark.scheduler._
-import spark.executor.TaskMetrics
+import org.apache.spark._
+import org.apache.spark.scheduler._
+import org.apache.spark.executor.TaskMetrics
 import java.nio.ByteBuffer
-import spark.util.FakeClock
+import org.apache.spark.util.FakeClock
 
 /**
  * A mock ClusterScheduler implementation that just remembers information about tasks started and
diff --git a/core/src/test/scala/spark/scheduler/cluster/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
similarity index 91%
rename from core/src/test/scala/spark/scheduler/cluster/FakeTask.scala
rename to core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
index de9e66be20ae6116828bae17650dd0d0bb06f768..2f12aaed18c6c99f485363b3d60e330feb795114 100644
--- a/core/src/test/scala/spark/scheduler/cluster/FakeTask.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
-import spark.scheduler.{TaskLocation, Task}
+import org.apache.spark.scheduler.{TaskLocation, Task}
 
 class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId) {
   override def run(attemptId: Long): Int = 0
diff --git a/core/src/test/scala/spark/scheduler/local/LocalSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/scheduler/local/LocalSchedulerSuite.scala
rename to core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
index d28ee47fa335ef81e512b943f674c8a4f6ea5a8d..111340a65ce1a9ea65950fd7c059bc133e96417a 100644
--- a/core/src/test/scala/spark/scheduler/local/LocalSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.scheduler.local
+package org.apache.spark.scheduler.local
 
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
 
-import spark._
-import spark.scheduler._
-import spark.scheduler.cluster._
+import org.apache.spark._
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster._
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.{ConcurrentMap, HashMap}
 import java.util.concurrent.Semaphore
diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
similarity index 97%
rename from core/src/test/scala/spark/storage/BlockManagerSuite.scala
rename to core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index b719d65342de6c25ef2ded153f9294f8a2dc7913..88ba10f2f2d2ede78c143071ab9530a7a45c1622 100644
--- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.storage
+package org.apache.spark.storage
 
 import java.nio.ByteBuffer
 
@@ -29,11 +29,12 @@ import org.scalatest.concurrent.Timeouts._
 import org.scalatest.matchers.ShouldMatchers._
 import org.scalatest.time.SpanSugar._
 
-import spark.JavaSerializer
-import spark.KryoSerializer
-import spark.SizeEstimator
-import spark.util.AkkaUtils
-import spark.util.ByteBufferInputStream
+import org.apache.spark.JavaSerializer
+import org.apache.spark.KryoSerializer
+import org.apache.spark.SizeEstimator
+import org.apache.spark.Utils
+import org.apache.spark.util.AkkaUtils
+import org.apache.spark.util.ByteBufferInputStream
 
 
 class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester {
@@ -56,7 +57,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     System.setProperty("spark.hostPort", "localhost:" + boundPort)
 
     master = new BlockManagerMaster(
-      actorSystem.actorOf(Props(new spark.storage.BlockManagerMasterActor(true))))
+      actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))
 
     // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case
     oldArch = System.setProperty("os.arch", "amd64")
@@ -65,7 +66,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     val initialize = PrivateMethod[Unit]('initialize)
     SizeEstimator invokePrivate initialize()
     // Set some value ...
-    System.setProperty("spark.hostPort", spark.Utils.localHostName() + ":" + 1111)
+    System.setProperty("spark.hostPort", Utils.localHostName() + ":" + 1111)
   }
 
   after {
@@ -105,10 +106,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     assert(level2 === level1, "level2 is not same as level1")
     assert(level2.eq(level1), "level2 is not the same object as level1")
     assert(level3 != level1, "level3 is same as level1")
-    val bytes1 = spark.Utils.serialize(level1)
-    val level1_ = spark.Utils.deserialize[StorageLevel](bytes1)
-    val bytes2 = spark.Utils.serialize(level2)
-    val level2_ = spark.Utils.deserialize[StorageLevel](bytes2)
+    val bytes1 = Utils.serialize(level1)
+    val level1_ = Utils.deserialize[StorageLevel](bytes1)
+    val bytes2 = Utils.serialize(level2)
+    val level2_ = Utils.deserialize[StorageLevel](bytes2)
     assert(level1_ === level1, "Deserialized level1 not same as original level1")
     assert(level1_.eq(level1), "Deserialized level1 not the same object as original level2")
     assert(level2_ === level2, "Deserialized level2 not same as original level2")
@@ -122,10 +123,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     assert(id2 === id1, "id2 is not same as id1")
     assert(id2.eq(id1), "id2 is not the same object as id1")
     assert(id3 != id1, "id3 is same as id1")
-    val bytes1 = spark.Utils.serialize(id1)
-    val id1_ = spark.Utils.deserialize[BlockManagerId](bytes1)
-    val bytes2 = spark.Utils.serialize(id2)
-    val id2_ = spark.Utils.deserialize[BlockManagerId](bytes2)
+    val bytes1 = Utils.serialize(id1)
+    val id1_ = Utils.deserialize[BlockManagerId](bytes1)
+    val bytes2 = Utils.serialize(id2)
+    val id2_ = Utils.deserialize[BlockManagerId](bytes2)
     assert(id1_ === id1, "Deserialized id1 is not same as original id1")
     assert(id1_.eq(id1), "Deserialized id1 is not the same object as original id1")
     assert(id2_ === id2, "Deserialized id2 is not same as original id2")
diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala
similarity index 98%
rename from core/src/test/scala/spark/ui/UISuite.scala
rename to core/src/test/scala/org/apache/spark/ui/UISuite.scala
index 735a794396fa247fe967b300d39ad3428e6e7556..3321fb5eb71b5315d207c6013324fc67d6f504ac 100644
--- a/core/src/test/scala/spark/ui/UISuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.ui
+package org.apache.spark.ui
 
 import scala.util.{Failure, Success, Try}
 import java.net.ServerSocket
diff --git a/core/src/test/scala/spark/util/DistributionSuite.scala b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala
similarity index 97%
rename from core/src/test/scala/spark/util/DistributionSuite.scala
rename to core/src/test/scala/org/apache/spark/util/DistributionSuite.scala
index 6578b55e8243ded61abba4a5bae94ca538cadaa8..63642461e4465a4d764120eeb5bdab2409295023 100644
--- a/core/src/test/scala/spark/util/DistributionSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import org.scalatest.FunSuite
 import org.scalatest.matchers.ShouldMatchers
diff --git a/core/src/test/scala/spark/util/FakeClock.scala b/core/src/test/scala/org/apache/spark/util/FakeClock.scala
similarity index 96%
rename from core/src/test/scala/spark/util/FakeClock.scala
rename to core/src/test/scala/org/apache/spark/util/FakeClock.scala
index 236706317e351c7b5fadbf90754473a82cc6e95d..0a45917b08dd250fcd3cafe3456592d5fb0fb152 100644
--- a/core/src/test/scala/spark/util/FakeClock.scala
+++ b/core/src/test/scala/org/apache/spark/util/FakeClock.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 class FakeClock extends Clock {
   private var time = 0L
diff --git a/core/src/test/scala/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala
similarity index 98%
rename from core/src/test/scala/spark/util/NextIteratorSuite.scala
rename to core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala
index fdbd43d941b1e645feb35f7ba70372c031a31643..45867463a5132a7a8a004372912b4b6896c6a0b9 100644
--- a/core/src/test/scala/spark/util/NextIteratorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import org.scalatest.FunSuite
 import org.scalatest.matchers.ShouldMatchers
diff --git a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/RateLimitedOutputStreamSuite.scala
similarity index 97%
rename from core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
rename to core/src/test/scala/org/apache/spark/util/RateLimitedOutputStreamSuite.scala
index 4c0044202f033d801270f6d47b17e72698ac011a..a9dd0b1a5b61554b8bf8eb0d5f22bfc9156707fa 100644
--- a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/RateLimitedOutputStreamSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.util
+package org.apache.spark.util
 
 import org.scalatest.FunSuite
 import java.io.ByteArrayOutputStream
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 84749fda4e6051921747bdbbf3ea086b14945960..349eb92a47f77d5c29e05e02a6533816b220d5fa 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -100,7 +100,7 @@
                                 <li><a href="tuning.html">Tuning Guide</a></li>
                                 <li><a href="hardware-provisioning.html">Hardware Provisioning</a></li>
                                 <li><a href="building-with-maven.html">Building Spark with Maven</a></li>
-                                <li><a href="contributing-to-spark.html">Contributing to Spark</a></li>
+                                <li><a href="contributing-to-org.apache.spark.html">Contributing to Spark</a></li>
                             </ul>
                         </li>
                     </ul>
diff --git a/examples/pom.xml b/examples/pom.xml
index 687fbcca8f34cf3b7b003fd2b8a2d3f72d71d032..13b553151137e3d4bac239b3a27a5631c7b57363 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -19,13 +19,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-examples</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Examples</name>
@@ -33,25 +33,25 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-core</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-mllib</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-bagel</artifactId>
       <version>${project.version}</version>
       <scope>provided</scope>
@@ -132,7 +132,7 @@
       <id>hadoop2-yarn</id>
       <dependencies>
         <dependency>
-          <groupId>org.spark-project</groupId>
+          <groupId>org.apache.spark</groupId>
           <artifactId>spark-yarn</artifactId>
           <version>${project.version}</version>
           <scope>provided</scope>
diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java
similarity index 94%
rename from examples/src/main/java/spark/examples/JavaHdfsLR.java
rename to examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java
index 9485e0cfa96639c9ac36d7eb5aca186fb63b3c3f..be0d38589c5df6f2925e06f7a081ff6c72057530 100644
--- a/examples/src/main/java/spark/examples/JavaHdfsLR.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.examples;
+package org.apache.spark.examples;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.Function;
-import spark.api.java.function.Function2;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
 
 import java.io.Serializable;
 import java.util.Arrays;
diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java
similarity index 92%
rename from examples/src/main/java/spark/examples/JavaKMeans.java
rename to examples/src/main/java/org/apache/spark/examples/JavaKMeans.java
index 2d34776177ca261d4ab82f56cc4d8e550be66cfc..5a6afe7eaefd6112a7348e77db4a40e1bc9136d9 100644
--- a/examples/src/main/java/spark/examples/JavaKMeans.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.examples;
+package org.apache.spark.examples;
 
 import scala.Tuple2;
-import spark.api.java.JavaPairRDD;
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.Function;
-import spark.api.java.function.PairFunction;
-import spark.util.Vector;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.util.Vector;
 
 import java.util.List;
 import java.util.Map;
diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
similarity index 94%
rename from examples/src/main/java/spark/examples/JavaLogQuery.java
rename to examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
index d22684d98044f69745d5e96fc112fb6544e57565..152f02921338a8ab974effdd211e1365f5f55741 100644
--- a/examples/src/main/java/spark/examples/JavaLogQuery.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package spark.examples;
+package org.apache.spark.examples;
 
 import com.google.common.collect.Lists;
 import scala.Tuple2;
 import scala.Tuple3;
-import spark.api.java.JavaPairRDD;
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.Function2;
-import spark.api.java.function.PairFunction;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
 
 import java.io.Serializable;
 import java.util.Collections;
diff --git a/examples/src/main/java/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
similarity index 89%
rename from examples/src/main/java/spark/examples/JavaPageRank.java
rename to examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
index 75df1af2e30b84fbc6ddd69b6189d8ebe2eeeedd..c5603a639bdd9c1abf3127c7ebf859b362db4aac 100644
--- a/examples/src/main/java/spark/examples/JavaPageRank.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package spark.examples;
+package org.apache.spark.examples;
 
 import scala.Tuple2;
-import spark.api.java.JavaPairRDD;
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.FlatMapFunction;
-import spark.api.java.function.Function;
-import spark.api.java.function.Function2;
-import spark.api.java.function.PairFlatMapFunction;
-import spark.api.java.function.PairFunction;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.api.java.function.PairFunction;
 
 import java.util.List;
 import java.util.ArrayList;
diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java
similarity index 89%
rename from examples/src/main/java/spark/examples/JavaSparkPi.java
rename to examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java
index d5f42fbb38e4698f7e555353481f6699603f1862..4a2380caf5af554c1daa13f198326a7961699e5c 100644
--- a/examples/src/main/java/spark/examples/JavaSparkPi.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.examples;
+package org.apache.spark.examples;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.Function;
-import spark.api.java.function.Function2;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
 
 import java.util.ArrayList;
 import java.util.List;
diff --git a/examples/src/main/java/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java
similarity index 94%
rename from examples/src/main/java/spark/examples/JavaTC.java
rename to examples/src/main/java/org/apache/spark/examples/JavaTC.java
index 559d7f9e53372a1345f29311e611f96c01002460..17f21f6b776d152cfff1c733542fd9afb2c1975f 100644
--- a/examples/src/main/java/spark/examples/JavaTC.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.examples;
+package org.apache.spark.examples;
 
 import scala.Tuple2;
-import spark.api.java.JavaPairRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.PairFunction;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.PairFunction;
 
 import java.util.ArrayList;
 import java.util.HashSet;
diff --git a/examples/src/main/java/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
similarity index 85%
rename from examples/src/main/java/spark/examples/JavaWordCount.java
rename to examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
index 1af370c1c3852478ef982f69f667cb635e677a60..07d32ad659a74dd1bcfcdc808be0022646b8bd67 100644
--- a/examples/src/main/java/spark/examples/JavaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.examples;
+package org.apache.spark.examples;
 
 import scala.Tuple2;
-import spark.api.java.JavaPairRDD;
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.FlatMapFunction;
-import spark.api.java.function.Function2;
-import spark.api.java.function.PairFunction;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
 
 import java.util.Arrays;
 import java.util.List;
diff --git a/examples/src/main/java/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
similarity index 88%
rename from examples/src/main/java/spark/mllib/examples/JavaALS.java
rename to examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
index b48f459cb7ee2d4f5c385ea5b10b58ad90c8b309..628cb892b686267c1996433faf324ab35b45ced4 100644
--- a/examples/src/main/java/spark/mllib/examples/JavaALS.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.mllib.examples;
+package org.apache.spark.mllib.examples;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.Function;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
 
-import spark.mllib.recommendation.ALS;
-import spark.mllib.recommendation.MatrixFactorizationModel;
-import spark.mllib.recommendation.Rating;
+import org.apache.spark.mllib.recommendation.ALS;
+import org.apache.spark.mllib.recommendation.MatrixFactorizationModel;
+import org.apache.spark.mllib.recommendation.Rating;
 
 import java.io.Serializable;
 import java.util.Arrays;
diff --git a/examples/src/main/java/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
similarity index 89%
rename from examples/src/main/java/spark/mllib/examples/JavaKMeans.java
rename to examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
index 02f40438b82c3667a076d4a51bf8c70e7aa29131..cd59a139b9fee00c5faa413d2ca9a453d1ba9526 100644
--- a/examples/src/main/java/spark/mllib/examples/JavaKMeans.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.mllib.examples;
+package org.apache.spark.mllib.examples;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.Function;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
 
-import spark.mllib.clustering.KMeans;
-import spark.mllib.clustering.KMeansModel;
+import org.apache.spark.mllib.clustering.KMeans;
+import org.apache.spark.mllib.clustering.KMeansModel;
 
 import java.util.Arrays;
 import java.util.StringTokenizer;
diff --git a/examples/src/main/java/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
similarity index 87%
rename from examples/src/main/java/spark/mllib/examples/JavaLR.java
rename to examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
index bf4aeaf40f63e411a39353b0e49a3d9aeffd3904..258061c8e6ba627fd388b4991a1be62d40eb3277 100644
--- a/examples/src/main/java/spark/mllib/examples/JavaLR.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package spark.mllib.examples;
+package org.apache.spark.mllib.examples;
 
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.Function;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
 
-import spark.mllib.classification.LogisticRegressionWithSGD;
-import spark.mllib.classification.LogisticRegressionModel;
-import spark.mllib.regression.LabeledPoint;
+import org.apache.spark.mllib.classification.LogisticRegressionWithSGD;
+import org.apache.spark.mllib.classification.LogisticRegressionModel;
+import org.apache.spark.mllib.regression.LabeledPoint;
 
 import java.util.Arrays;
 import java.util.StringTokenizer;
diff --git a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
similarity index 90%
rename from examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
rename to examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
index 096a9ae21976ffed603329102ba056acf1b0ccb4..261813bf2f39c49f61df463e864df12fdedad858 100644
--- a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples;
+package org.apache.spark.streaming.examples;
 
-import spark.api.java.function.Function;
-import spark.streaming.*;
-import spark.streaming.api.java.*;
-import spark.streaming.dstream.SparkFlumeEvent;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.streaming.*;
+import org.apache.spark.streaming.api.java.*;
+import org.apache.spark.streaming.dstream.SparkFlumeEvent;
 
 /**
  *  Produces a count of events received from Flume.
diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
similarity index 86%
rename from examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
rename to examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
index c54d3f3d599a5e58a9f52901e30e79f45741a75b..def87c199be57eb0352c35fc5e3858e8ea46f7c7 100644
--- a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples;
+package org.apache.spark.streaming.examples;
 
 import com.google.common.collect.Lists;
 import scala.Tuple2;
-import spark.api.java.function.FlatMapFunction;
-import spark.api.java.function.Function2;
-import spark.api.java.function.PairFunction;
-import spark.streaming.Duration;
-import spark.streaming.api.java.JavaDStream;
-import spark.streaming.api.java.JavaPairDStream;
-import spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
 
 /**
  * Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
diff --git a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
similarity index 85%
rename from examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
rename to examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
index 1f4a991542876e9b1af7615eaf445af05fe19207..c8c7389dd1bbaa11407f4d28ea1bbf08925c5b40 100644
--- a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples;
+package org.apache.spark.streaming.examples;
 
 import com.google.common.collect.Lists;
 import scala.Tuple2;
-import spark.api.java.JavaRDD;
-import spark.api.java.function.Function2;
-import spark.api.java.function.PairFunction;
-import spark.streaming.Duration;
-import spark.streaming.api.java.JavaDStream;
-import spark.streaming.api.java.JavaPairDStream;
-import spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.streaming.Duration;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
 
 import java.util.LinkedList;
 import java.util.List;
diff --git a/examples/src/main/scala/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
similarity index 95%
rename from examples/src/main/scala/spark/examples/BroadcastTest.scala
rename to examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
index 911490cb6c12412dc24d047c1a6dad5fe4b28331..868ff81f67a82837ed0c108374c009d37a9a75bb 100644
--- a/examples/src/main/scala/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark.SparkContext
+import org.apache.spark.SparkContext
 
 object BroadcastTest {
   def main(args: Array[String]) {
diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala
similarity index 98%
rename from examples/src/main/scala/spark/examples/CassandraTest.scala
rename to examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala
index 104bfd52047f30411460d11544db543aacbf9082..33bf7151a7cc493bfce74029a3734bbc138fbd9a 100644
--- a/examples/src/main/scala/spark/examples/CassandraTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import org.apache.hadoop.mapreduce.Job
 import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat
 import org.apache.cassandra.hadoop.ConfigHelper
 import org.apache.cassandra.hadoop.ColumnFamilyInputFormat
 import org.apache.cassandra.thrift._
-import spark.SparkContext
-import spark.SparkContext._
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
 import java.nio.ByteBuffer
 import java.util.SortedMap
 import org.apache.cassandra.db.IColumn
diff --git a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
similarity index 94%
rename from examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
rename to examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
index 67ddaec8d200355b6d443d417e474d53a709582c..92eb96bd8e0c2a04f5a26adbad79cc2a757f2be0 100644
--- a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark.SparkContext
+import org.apache.spark.SparkContext
 
 object ExceptionHandlingTest {
   def main(args: Array[String]) {
diff --git a/examples/src/main/scala/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
similarity index 94%
rename from examples/src/main/scala/spark/examples/GroupByTest.scala
rename to examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
index 5cee4136158b45b68a7b2ea9a785e50478d7c3e1..42c2e0e8e19c4af07e28ea8cd19bda9b935b871b 100644
--- a/examples/src/main/scala/spark/examples/GroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark.SparkContext
-import spark.SparkContext._
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
 import java.util.Random
 
 object GroupByTest {
diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
similarity index 94%
rename from examples/src/main/scala/spark/examples/HBaseTest.scala
rename to examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
index 4dd6c243acfa28962e5d675b00bda59f7d334458..efe2e93b0dc91cb7a77c532ca1084bc6fca3d593 100644
--- a/examples/src/main/scala/spark/examples/HBaseTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark._
-import spark.rdd.NewHadoopRDD
+import org.apache.spark._
+import org.apache.spark.rdd.NewHadoopRDD
 import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor}
 import org.apache.hadoop.hbase.client.HBaseAdmin
 import org.apache.hadoop.hbase.mapreduce.TableInputFormat
diff --git a/examples/src/main/scala/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
similarity index 95%
rename from examples/src/main/scala/spark/examples/HdfsTest.scala
rename to examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
index 23258336e23a1aa1e37731271a6fdca0caab9b38..d6a88d3032c494bbf2bf5f651a0041993fe09b79 100644
--- a/examples/src/main/scala/spark/examples/HdfsTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark._
+import org.apache.spark._
 
 object HdfsTest {
   def main(args: Array[String]) {
diff --git a/examples/src/main/scala/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
similarity index 99%
rename from examples/src/main/scala/spark/examples/LocalALS.scala
rename to examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
index 7a449a9d720185d151ae3eb604913f68194273ab..4af45b2b4a0670df6d8c5072c818cdf0be72fe1b 100644
--- a/examples/src/main/scala/spark/examples/LocalALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import scala.math.sqrt
 import cern.jet.math._
diff --git a/examples/src/main/scala/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala
similarity index 96%
rename from examples/src/main/scala/spark/examples/LocalFileLR.scala
rename to examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala
index c1f8d32aa87b9852662c896107c163d8d3fec491..fb130ea1988f76ff9456e5f2d2db58b1e53377cc 100644
--- a/examples/src/main/scala/spark/examples/LocalFileLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import java.util.Random
-import spark.util.Vector
+import org.apache.spark.util.Vector
 
 object LocalFileLR {
   val D = 10   // Numer of dimensions
diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala
similarity index 96%
rename from examples/src/main/scala/spark/examples/LocalKMeans.scala
rename to examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala
index 0a0bc6f476b41496b804c620f1091f9f48560619..f90ea35cd447c035848fbf124d726d94645ade74 100644
--- a/examples/src/main/scala/spark/examples/LocalKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import java.util.Random
-import spark.util.Vector
-import spark.SparkContext._
+import org.apache.spark.util.Vector
+import org.apache.spark.SparkContext._
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
 
diff --git a/examples/src/main/scala/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala
similarity index 96%
rename from examples/src/main/scala/spark/examples/LocalLR.scala
rename to examples/src/main/scala/org/apache/spark/examples/LocalLR.scala
index ab99bf1fbe44bbc240dd46186167f673fc1a0b7b..cd4e9f1af0e2c81887ae8b579de3ff19960e415c 100644
--- a/examples/src/main/scala/spark/examples/LocalLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import java.util.Random
-import spark.util.Vector
+import org.apache.spark.util.Vector
 
 /**
  * Logistic regression based classification.
diff --git a/examples/src/main/scala/spark/examples/LocalPi.scala b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala
similarity index 94%
rename from examples/src/main/scala/spark/examples/LocalPi.scala
rename to examples/src/main/scala/org/apache/spark/examples/LocalPi.scala
index ccd69695df4ad1a8edca6cd2f43d33b0e8a1942c..bb7f22ec8df42b489f1cf1ff81129f323f941d4a 100644
--- a/examples/src/main/scala/spark/examples/LocalPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import scala.math.random
-import spark._
+import org.apache.spark._
 import SparkContext._
 
 object LocalPi {
diff --git a/examples/src/main/scala/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
similarity index 97%
rename from examples/src/main/scala/spark/examples/LogQuery.scala
rename to examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
index e815ececf7ae95461c3abe886014a87870b91869..17ff3ce76497f6297b583e3678c9ac06fcc9550a 100644
--- a/examples/src/main/scala/spark/examples/LogQuery.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark.SparkContext
-import spark.SparkContext._
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
 /**
  * Executes a roll up-style query against Apache logs.
  */
diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
similarity index 95%
rename from examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
rename to examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
index d0b1cf06e59df22e76fad4d3cc992c7250f362ab..f79f0142b8679134210db7616284176a994f3c36 100644
--- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark.SparkContext
+import org.apache.spark.SparkContext
 
 object MultiBroadcastTest {
   def main(args: Array[String]) {
diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
similarity index 96%
rename from examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
rename to examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
index d197bbaf7c47837fc8376e7e3445f7fd7b13ae29..37ddfb5db7635ca2d7303b77105b74580a515efb 100644
--- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark.SparkContext
-import spark.SparkContext._
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
 import java.util.Random
 
 object SimpleSkewedGroupByTest {
diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
similarity index 95%
rename from examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
rename to examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
index 4641b8244487eb40e022fd70159d97d308353a7a..9c954b2b5baa996a0cdf433fe7c373371cdfd60c 100644
--- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark.SparkContext
-import spark.SparkContext._
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
 import java.util.Random
 
 object SkewedGroupByTest {
diff --git a/examples/src/main/scala/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
similarity index 98%
rename from examples/src/main/scala/spark/examples/SparkALS.scala
rename to examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
index ba0dfd8f9b5c4109df77a38cd168f808f4ec93e8..814944ba1c6bf63f591ae3ca04b3e0b864f3fef8 100644
--- a/examples/src/main/scala/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import scala.math.sqrt
 import cern.jet.math._
 import cern.colt.matrix._
 import cern.colt.matrix.linalg._
-import spark._
+import org.apache.spark._
 
 /**
  * Alternating least squares matrix factorization.
diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
similarity index 94%
rename from examples/src/main/scala/spark/examples/SparkHdfsLR.scala
rename to examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
index 43c91156640c899aaf622c87efdbe646b4adf398..646682878fda5373027b660c53c17e39176d0416 100644
--- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import java.util.Random
 import scala.math.exp
-import spark.util.Vector
-import spark._
-import spark.scheduler.InputFormatInfo
+import org.apache.spark.util.Vector
+import org.apache.spark._
+import org.apache.spark.scheduler.InputFormatInfo
 
 /**
  * Logistic regression based classification.
diff --git a/examples/src/main/scala/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
similarity index 94%
rename from examples/src/main/scala/spark/examples/SparkKMeans.scala
rename to examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
index 38ed3b149af2051bbf1426a8f1031680ac37c2e8..f7bf75b4e5038c10e374ae6e26d4e561b2cda568 100644
--- a/examples/src/main/scala/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import java.util.Random
-import spark.SparkContext
-import spark.util.Vector
-import spark.SparkContext._
+import org.apache.spark.SparkContext
+import org.apache.spark.util.Vector
+import org.apache.spark.SparkContext._
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
 
diff --git a/examples/src/main/scala/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
similarity index 95%
rename from examples/src/main/scala/spark/examples/SparkLR.scala
rename to examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
index 52a0d69744ad6005cde158c9f61868cd86e708bf..9ed9fe4d761d5f8ab7a58b9b98137fe59085e779 100644
--- a/examples/src/main/scala/spark/examples/SparkLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import java.util.Random
 import scala.math.exp
-import spark.util.Vector
-import spark._
+import org.apache.spark.util.Vector
+import org.apache.spark._
 
 /**
  * Logistic regression based classification.
diff --git a/examples/src/main/scala/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
similarity index 91%
rename from examples/src/main/scala/spark/examples/SparkPageRank.scala
rename to examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
index dedbbd01a3a9c3b0ba6d181721e1e3dfb3812d1f..2721caf08bd176ca61ff3c10a4d285528c2cd179 100644
--- a/examples/src/main/scala/spark/examples/SparkPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
@@ -1,7 +1,7 @@
-package spark.examples
+package org.apache.spark.examples
 
-import spark.SparkContext._
-import spark.SparkContext
+import org.apache.spark.SparkContext._
+import org.apache.spark.SparkContext
 
 
 /**
diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
similarity index 96%
rename from examples/src/main/scala/spark/examples/SparkPi.scala
rename to examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
index 00560ac9d160a6a3c4e6e55a5423fcd985a8ea53..5a2bc9b0d08eebc70a35731b593e1e7dd5f7e6b7 100644
--- a/examples/src/main/scala/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
 import scala.math.random
-import spark._
+import org.apache.spark._
 import SparkContext._
 
 /** Computes an approximation to pi */
diff --git a/examples/src/main/scala/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
similarity index 97%
rename from examples/src/main/scala/spark/examples/SparkTC.scala
rename to examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
index bf988a953bbb36187ff31a63b76510a6d9324c5d..5a7a9d1bd8f744bcc92c5e8f0a30c50051fd64bc 100644
--- a/examples/src/main/scala/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.examples
+package org.apache.spark.examples
 
-import spark._
+import org.apache.spark._
 import SparkContext._
 import scala.util.Random
 import scala.collection.mutable
diff --git a/examples/src/main/scala/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala
similarity index 95%
rename from examples/src/main/scala/spark/examples/bagel/PageRankUtils.scala
rename to examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala
index c23ee9895f2b73f139a551ce542d2da1dbfafe7f..b190e83c4d36a86738824009559f8dfa3948949d 100644
--- a/examples/src/main/scala/spark/examples/bagel/PageRankUtils.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.examples.bagel
+package org.apache.spark.examples.bagel
 
-import spark._
-import spark.SparkContext._
+import org.apache.spark._
+import org.apache.spark.SparkContext._
 
-import spark.bagel._
-import spark.bagel.Bagel._
+import org.apache.spark.bagel._
+import org.apache.spark.bagel.Bagel._
 
 import scala.collection.mutable.ArrayBuffer
 
diff --git a/examples/src/main/scala/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
similarity index 93%
rename from examples/src/main/scala/spark/examples/bagel/WikipediaPageRank.scala
rename to examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
index 00635a7ffa2f7471c3db86fe3d31d1aea91d9de4..b1f606e48e03264f97474817eb6a1e1fd85ef1f7 100644
--- a/examples/src/main/scala/spark/examples/bagel/WikipediaPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.examples.bagel
+package org.apache.spark.examples.bagel
 
-import spark._
-import spark.SparkContext._
+import org.apache.spark._
+import org.apache.spark.SparkContext._
 
-import spark.bagel._
-import spark.bagel.Bagel._
+import org.apache.spark.bagel._
+import org.apache.spark.bagel.Bagel._
 
 import scala.xml.{XML,NodeSeq}
 
@@ -37,7 +37,7 @@ object WikipediaPageRank {
       System.exit(-1)
     }
 
-    System.setProperty("spark.serializer", "spark.KryoSerializer")
+    System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer")
     System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName)
 
     val inputFile = args(0)
diff --git a/examples/src/main/scala/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
similarity index 96%
rename from examples/src/main/scala/spark/examples/bagel/WikipediaPageRankStandalone.scala
rename to examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
index c416ddbc584bdb2d20b9b5b5abb55231762f325d..3bfa48eaf3d19275b5db80e9ad70555362243027 100644
--- a/examples/src/main/scala/spark/examples/bagel/WikipediaPageRankStandalone.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.examples.bagel
+package org.apache.spark.examples.bagel
 
-import spark._
+import org.apache.spark._
 import serializer.{DeserializationStream, SerializationStream, SerializerInstance}
-import spark.SparkContext._
+import org.apache.spark.SparkContext._
 
-import spark.bagel._
-import spark.bagel.Bagel._
+import org.apache.spark.bagel._
+import org.apache.spark.bagel.Bagel._
 
 import scala.xml.{XML,NodeSeq}
 
@@ -131,7 +131,7 @@ object WikipediaPageRankStandalone {
   }
 }
 
-class WPRSerializer extends spark.serializer.Serializer {
+class WPRSerializer extends org.apache.spark.serializer.Serializer {
   def newInstance(): SerializerInstance = new WPRSerializerInstance()
 }
 
diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
similarity index 93%
rename from examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
index 05d3176478130301d34db93c81de5c5d268ee279..cd3423a07b6b788fb3b7bc6f2c9e8c22ffdc415c 100644
--- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
 import scala.collection.mutable.LinkedList
 import scala.util.Random
@@ -25,11 +25,11 @@ import akka.actor.ActorRef
 import akka.actor.Props
 import akka.actor.actorRef2Scala
 
-import spark.streaming.Seconds
-import spark.streaming.StreamingContext
-import spark.streaming.StreamingContext.toPairDStreamFunctions
-import spark.streaming.receivers.Receiver
-import spark.util.AkkaUtils
+import org.apache.spark.streaming.Seconds
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions
+import org.apache.spark.streaming.receivers.Receiver
+import org.apache.spark.util.AkkaUtils
 
 case class SubscribeReceiver(receiverActor: ActorRef)
 case class UnsubscribeReceiver(receiverActor: ActorRef)
@@ -80,7 +80,7 @@ class FeederActor extends Actor {
  * goes and subscribe to a typical publisher/feeder actor and receives
  * data.
  *
- * @see [[spark.streaming.examples.FeederActor]]
+ * @see [[org.apache.spark.streaming.examples.FeederActor]]
  */
 class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String)
 extends Actor with Receiver {
diff --git a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
similarity index 92%
rename from examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
index 3ab4fc2c37b458381fefadd9b530a241b694800a..9f6e163454a64fe062989c7a5794f9ffd07f165e 100644
--- a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
-import spark.util.IntParam
-import spark.storage.StorageLevel
-import spark.streaming._
+import org.apache.spark.util.IntParam
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming._
 
 /**
  *  Produces a count of events received from Flume.
diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
similarity index 92%
rename from examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
index 30af01a26f42e4243fde65782dda231134d45c9e..bc8564b3ba080aa3ed3f367fb1211736d9035411 100644
--- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
-import spark.streaming.{Seconds, StreamingContext}
-import spark.streaming.StreamingContext._
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
 
 
 /**
diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
similarity index 92%
rename from examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
index d9c76d1a33bf6c1da810bd4d4fca354d45fa2518..12f939d5a7e4b529c05c8dfe77d67e8c3f254b40 100644
--- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
 import java.util.Properties
 import kafka.message.Message
 import kafka.producer.SyncProducerConfig
 import kafka.producer._
-import spark.SparkContext
-import spark.streaming._
-import spark.streaming.StreamingContext._
-import spark.storage.StorageLevel
-import spark.streaming.util.RawTextHelper._
+import org.apache.spark.SparkContext
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.util.RawTextHelper._
 
 /**
  * Consumes messages from one or more topics in Kafka and does wordcount.
diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
similarity index 93%
rename from examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
index b29d79aac53d7fd71d1f9de36452a992eb7999e7..e2487dca5f2cc894098c2e1bb08aee761c85bbea 100644
--- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
-import spark.streaming.{Seconds, StreamingContext}
-import spark.streaming.StreamingContext._
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
 
 /**
  * Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
similarity index 90%
rename from examples/src/main/scala/spark/streaming/examples/QueueStream.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
index da36c8c23c65f3df19dcd0d1fd72e0398cab2ab4..822da8c9b5cccce5ac8106577139eb97112fa849 100644
--- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
-import spark.RDD
-import spark.streaming.{Seconds, StreamingContext}
-import spark.streaming.StreamingContext._
+import org.apache.spark.RDD
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
 
 import scala.collection.mutable.SynchronizedQueue
 
diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
similarity index 90%
rename from examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
index 7fb680bcc310ae47db2c8e5b76eca0dec8969f64..2e3d9ccf004793e707090e62967489b271fe71b0 100644
--- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
@@ -15,20 +15,20 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
-import spark.util.IntParam
-import spark.storage.StorageLevel
+import org.apache.spark.util.IntParam
+import org.apache.spark.storage.StorageLevel
 
-import spark.streaming._
-import spark.streaming.util.RawTextHelper
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.util.RawTextHelper
 
 /**
  * Receives text from multiple rawNetworkStreams and counts how many '\n' delimited
  * lines have the word 'the' in them. This is useful for benchmarking purposes. This
  * will only work with spark.streaming.util.RawTextSender running on all worker nodes
  * and with Spark using Kryo serialization (set Java property "spark.serializer" to
- * "spark.KryoSerializer").
+ * "org.apache.spark.KryoSerializer").
  * Usage: RawNetworkGrep <master> <numStreams> <host> <port> <batchMillis>
  *   <master> is the Spark master URL
  *   <numStream> is the number rawNetworkStreams, which should be same as number
diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
similarity index 95%
rename from examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
index b709fc3c87ca1487163cafcf4b9232674fea5f10..cb30c4edb30ce408faff73dc504c50f3a426f882 100644
--- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
-import spark.streaming._
-import spark.streaming.StreamingContext._
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.StreamingContext._
 
 /**
  * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every second.
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
similarity index 94%
rename from examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
index 8770abd57e02d72f6aa94926ea09ee94364cdcaf..35b6329ab3152caca3dd7564a88448cf2e9d284f 100644
--- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
-import spark.streaming.{Seconds, StreamingContext}
-import spark.storage.StorageLevel
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.storage.StorageLevel
 import com.twitter.algebird._
-import spark.streaming.StreamingContext._
-import spark.SparkContext._
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.SparkContext._
 
 /**
  * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
similarity index 94%
rename from examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
index cba5c986beac62e811c1c0918d1f0ac5b7d8a7d0..8bfde2a8297c1a133c80d3be08f8ad6ae0b145f3 100644
--- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
-import spark.streaming.{Seconds, StreamingContext}
-import spark.storage.StorageLevel
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.storage.StorageLevel
 import com.twitter.algebird.HyperLogLog._
 import com.twitter.algebird.HyperLogLogMonoid
-import spark.streaming.dstream.TwitterInputDStream
+import org.apache.spark.streaming.dstream.TwitterInputDStream
 
 /**
  * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
similarity index 94%
rename from examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
index 682b99f75e5ec6b9082a3f2f9c71c0fd29c1e50e..27aa6b14bf221ab08752ce721a2548e85e67f98b 100644
--- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
-import spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.{Seconds, StreamingContext}
 import StreamingContext._
-import spark.SparkContext._
+import org.apache.spark.SparkContext._
 
 /**
  * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter
diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
similarity index 95%
rename from examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
index a0cae06c3093fbae6bf73fad450a72390b6c640d..c8743b9e25523f70574fc01d8e3ab232c5c0eb97 100644
--- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples
+package org.apache.spark.streaming.examples
 
 import akka.actor.ActorSystem
 import akka.actor.actorRef2Scala
 import akka.zeromq._
-import spark.streaming.{ Seconds, StreamingContext }
-import spark.streaming.StreamingContext._
+import org.apache.spark.streaming.{ Seconds, StreamingContext }
+import org.apache.spark.streaming.StreamingContext._
 import akka.zeromq.Subscribe
 
 /**
diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
similarity index 98%
rename from examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
index dd36bbbf3207ccc8823b0df989ba546c0fac37f0..884d6d6f3414c7c18d511b45e1c8ea6618fee8be 100644
--- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples.clickstream
+package org.apache.spark.streaming.examples.clickstream
 
 import java.net.{InetAddress,ServerSocket,Socket,SocketException}
 import java.io.{InputStreamReader, BufferedReader, PrintWriter}
diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
similarity index 95%
rename from examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
rename to examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
index 152da2348989051fb00fb3d68a25953163975d1c..8282cc9269c13fc287e0e7dc38c0e921139efce5 100644
--- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.examples.clickstream
+package org.apache.spark.streaming.examples.clickstream
 
-import spark.streaming.{Seconds, StreamingContext}
-import spark.streaming.StreamingContext._
-import spark.SparkContext._
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.SparkContext._
 
 /** Analyses a streaming dataset of web page views. This class demonstrates several types of
   * operators available in Spark streaming.
diff --git a/mllib/pom.xml b/mllib/pom.xml
index ab31d5734e42a249e83f3b97c751c2d3fc981c98..2d5d3c00d10b426b8cbeda9f3716bd2e9b092dbf 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -19,13 +19,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-mllib</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project ML Library</name>
@@ -33,7 +33,7 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-core</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala
similarity index 88%
rename from mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala
index 70fae8c15a27549a7f935fc4b655fd571cff526e..4f4a7f5296ee63e157df7b0804e3d15138c92a86 100644
--- a/mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala
@@ -1,6 +1,6 @@
-package spark.mllib.classification
+package org.apache.spark.mllib.classification
 
-import spark.RDD
+import org.apache.spark.RDD
 
 trait ClassificationModel extends Serializable {
   /**
diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
similarity index 96%
rename from mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
index 482e4a674579474bb54f0f5d4c7c1df22b597e1f..91bb50c829c179a9808d78038741dc24977871c2 100644
--- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.mllib.classification
+package org.apache.spark.mllib.classification
 
 import scala.math.round
 
-import spark.{Logging, RDD, SparkContext}
-import spark.mllib.optimization._
-import spark.mllib.regression._
-import spark.mllib.util.MLUtils
-import spark.mllib.util.DataValidators
+import org.apache.spark.{Logging, RDD, SparkContext}
+import org.apache.spark.mllib.optimization._
+import org.apache.spark.mllib.regression._
+import org.apache.spark.mllib.util.MLUtils
+import org.apache.spark.mllib.util.DataValidators
 
 import org.jblas.DoubleMatrix
 
diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
similarity index 95%
rename from mllib/src/main/scala/spark/mllib/classification/SVM.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
index 69393cd7b07982522a65fda44d0a5fb335598ff1..c92c7cc3f34bd5f1af7354b4ffaf0f495b5b52d5 100644
--- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.mllib.classification
+package org.apache.spark.mllib.classification
 
 import scala.math.signum
 
-import spark.{Logging, RDD, SparkContext}
-import spark.mllib.optimization._
-import spark.mllib.regression._
-import spark.mllib.util.MLUtils
-import spark.mllib.util.DataValidators
+import org.apache.spark.{Logging, RDD, SparkContext}
+import org.apache.spark.mllib.optimization._
+import org.apache.spark.mllib.regression._
+import org.apache.spark.mllib.util.MLUtils
+import org.apache.spark.mllib.util.DataValidators
 
 import org.jblas.DoubleMatrix
 
diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
similarity index 98%
rename from mllib/src/main/scala/spark/mllib/clustering/KMeans.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
index 97e3d110ae5e3a44c34abd49dd496dddd7aaa014..2c3db099fa8f08a66a26e24033918e41c94ad56c 100644
--- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.mllib.clustering
+package org.apache.spark.mllib.clustering
 
 import scala.collection.mutable.ArrayBuffer
 import scala.util.Random
 
-import spark.{SparkContext, RDD}
-import spark.SparkContext._
-import spark.Logging
-import spark.mllib.util.MLUtils
+import org.apache.spark.{SparkContext, RDD}
+import org.apache.spark.SparkContext._
+import org.apache.spark.Logging
+import org.apache.spark.mllib.util.MLUtils
 
 import org.jblas.DoubleMatrix
 
diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
similarity index 90%
rename from mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
index b8f80e80cd7602e0b6f03e684443761fc74d9773..d1fe5d138d31cf221e9cdfd1bd4818f3eb837d51 100644
--- a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.mllib.clustering
+package org.apache.spark.mllib.clustering
 
-import spark.RDD
-import spark.SparkContext._
-import spark.mllib.util.MLUtils
+import org.apache.spark.RDD
+import org.apache.spark.SparkContext._
+import org.apache.spark.mllib.util.MLUtils
 
 
 /**
diff --git a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala
similarity index 98%
rename from mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala
index 89fe7d7e85391dcade62ec5ec9d45bda8360aecd..baf8251d8fc53df4e030b6d4d03de7f0b58a7e45 100644
--- a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.clustering
+package org.apache.spark.mllib.clustering
 
 import scala.util.Random
 
diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
similarity index 98%
rename from mllib/src/main/scala/spark/mllib/optimization/Gradient.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
index 05568f55af06077a6d10e9182d3a6eaf9485c30e..749e7364f4de1afde6371cec3cf5f9515bfd50d1 100644
--- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.optimization
+package org.apache.spark.mllib.optimization
 
 import org.jblas.DoubleMatrix
 
diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
similarity index 97%
rename from mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
index 31917df7e87124a9873b238481452fbd1bc078af..b62c9b3340e3f4c41b437b9d7cec5800bc4f5453 100644
--- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.mllib.optimization
+package org.apache.spark.mllib.optimization
 
-import spark.{Logging, RDD, SparkContext}
-import spark.SparkContext._
+import org.apache.spark.{Logging, RDD, SparkContext}
+import org.apache.spark.SparkContext._
 
 import org.jblas.DoubleMatrix
 
diff --git a/mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala
similarity index 93%
rename from mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala
index 76a519c338847ea12a87168057516c7c58197fc3..50059d385d2b2be5ea67bd08abb929da4a3c2cb6 100644
--- a/mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.mllib.optimization
+package org.apache.spark.mllib.optimization
 
-import spark.RDD
+import org.apache.spark.RDD
 
 trait Optimizer {
 
diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala
similarity index 98%
rename from mllib/src/main/scala/spark/mllib/optimization/Updater.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala
index db67d6b0bcd7d7c11d7c0fdc08d72c710e00ba73..4c51f4f881f76e1f02b8342219a1fd94ffed3a9c 100644
--- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.optimization
+package org.apache.spark.mllib.optimization
 
 import scala.math._
 import org.jblas.DoubleMatrix
diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
similarity index 98%
rename from mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
index dbfbf5997540501012afa8b6ab97b74b988673ce..218217acfe5d635aabff81a09b007aba5a767e45 100644
--- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package spark.mllib.recommendation
+package org.apache.spark.mllib.recommendation
 
 import scala.collection.mutable.{ArrayBuffer, BitSet}
 import scala.util.Random
 import scala.util.Sorting
 
-import spark.{HashPartitioner, Partitioner, SparkContext, RDD}
-import spark.storage.StorageLevel
-import spark.KryoRegistrator
-import spark.SparkContext._
+import org.apache.spark.{HashPartitioner, Partitioner, SparkContext, RDD}
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.KryoRegistrator
+import org.apache.spark.SparkContext._
 
 import com.esotericsoftware.kryo.Kryo
 import org.jblas.{DoubleMatrix, SimpleBlas, Solve}
@@ -432,7 +432,7 @@ object ALS {
     val (master, ratingsFile, rank, iters, outputDir) =
       (args(0), args(1), args(2).toInt, args(3).toInt, args(4))
     val blocks = if (args.length == 6) args(5).toInt else -1
-    System.setProperty("spark.serializer", "spark.KryoSerializer")
+    System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer")
     System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName)
     System.setProperty("spark.kryo.referenceTracking", "false")
     System.setProperty("spark.kryoserializer.buffer.mb", "8")
diff --git a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
similarity index 94%
rename from mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
index 5e21717da525f5d9b89b819449670a931e20b497..ae9fe48aec3ef943c95b40c6c17b8e07e14d78b5 100644
--- a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.mllib.recommendation
+package org.apache.spark.mllib.recommendation
 
-import spark.RDD
-import spark.SparkContext._
+import org.apache.spark.RDD
+import org.apache.spark.SparkContext._
 
 import org.jblas._
 
diff --git a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
similarity index 96%
rename from mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
index d164d415d6a4005e73f3b8267d9be204ce2baff6..06015110ac0c3c5b2b1d98d524dd921339bdca79 100644
--- a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression
+package org.apache.spark.mllib.regression
 
-import spark.{Logging, RDD, SparkException}
-import spark.mllib.optimization._
+import org.apache.spark.{Logging, RDD, SparkException}
+import org.apache.spark.mllib.optimization._
 
 import org.jblas.DoubleMatrix
 
@@ -52,7 +52,7 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept:
    * @param testData RDD representing data points to be predicted
    * @return RDD[Double] where each entry contains the corresponding prediction
    */
-  def predict(testData: spark.RDD[Array[Double]]): RDD[Double] = {
+  def predict(testData: RDD[Array[Double]]): RDD[Double] = {
     // A small optimization to avoid serializing the entire model. Only the weightsMatrix
     // and intercept is needed.
     val localWeights = weightsMatrix
diff --git a/mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala
similarity index 96%
rename from mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala
index 3de60482c54cf75bbba7241c90c8c5041826cbc0..63240e24dc29c2999b06a8753b1700969851c6ea 100644
--- a/mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression
+package org.apache.spark.mllib.regression
 
 /**
  * Class that represents the features and labels of a data point.
diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
similarity index 97%
rename from mllib/src/main/scala/spark/mllib/regression/Lasso.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
index 0f33456ef435758a815a20bc5fbb8a6531718e2f..df3beb1959d11ca848281cb31919418d23dc6982 100644
--- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression
+package org.apache.spark.mllib.regression
 
-import spark.{Logging, RDD, SparkContext}
-import spark.mllib.optimization._
-import spark.mllib.util.MLUtils
+import org.apache.spark.{Logging, RDD, SparkContext}
+import org.apache.spark.mllib.optimization._
+import org.apache.spark.mllib.util.MLUtils
 
 import org.jblas.DoubleMatrix
 
diff --git a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
similarity index 97%
rename from mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
index 885ff5a30d423ade13058f3f1d8af63e32bb32b7..71f968471cdcc39c105ffb03028bae4f29b2f504 100644
--- a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression
+package org.apache.spark.mllib.regression
 
-import spark.{Logging, RDD, SparkContext}
-import spark.mllib.optimization._
-import spark.mllib.util.MLUtils
+import org.apache.spark.{Logging, RDD, SparkContext}
+import org.apache.spark.mllib.optimization._
+import org.apache.spark.mllib.util.MLUtils
 
 import org.jblas.DoubleMatrix
 
diff --git a/mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala
similarity index 95%
rename from mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala
index b845ba1a890ce3fcd0f218818a1ad922c648b574..8dd325efc08e3d82c8b50734dfb80c3f0a596f2d 100644
--- a/mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression
+package org.apache.spark.mllib.regression
 
-import spark.RDD
+import org.apache.spark.RDD
 
 trait RegressionModel extends Serializable {
   /**
diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
similarity index 97%
rename from mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
index cb1303dd99ed3848b3377bc4c941492771436253..228ab9e4e8f3bb240a90786fb4eabab08e9d55c4 100644
--- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression
+package org.apache.spark.mllib.regression
 
-import spark.{Logging, RDD, SparkContext}
-import spark.mllib.optimization._
-import spark.mllib.util.MLUtils
+import org.apache.spark.{Logging, RDD, SparkContext}
+import org.apache.spark.mllib.optimization._
+import org.apache.spark.mllib.util.MLUtils
 
 import org.jblas.DoubleMatrix
 
diff --git a/mllib/src/main/scala/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala
similarity index 91%
rename from mllib/src/main/scala/spark/mllib/util/DataValidators.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala
index 57553accf1fc5f1a5a5d4d1253ee5c6da0c5acc0..7fd46230716883c8cb22d0cf3096f44b46f3ac2b 100644
--- a/mllib/src/main/scala/spark/mllib/util/DataValidators.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.mllib.util
+package org.apache.spark.mllib.util
 
-import spark.{RDD, Logging}
-import spark.mllib.regression.LabeledPoint
+import org.apache.spark.{RDD, Logging}
+import org.apache.spark.mllib.regression.LabeledPoint
 
 /**
  * A collection of methods used to validate data before applying ML algorithms.
diff --git a/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala
similarity index 97%
rename from mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala
index 672b63f65ab9d35681ab0d17eee1d15bda481a5f..6500d47183e16710d3753c0da6ee7c0e0464e0a2 100644
--- a/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.mllib.util
+package org.apache.spark.mllib.util
 
 import scala.util.Random
 
-import spark.{RDD, SparkContext}
+import org.apache.spark.{RDD, SparkContext}
 
 /**
  * Generate test data for KMeans. This class first chooses k cluster centers
diff --git a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
similarity index 95%
rename from mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
index 9f48477f84ab019f4a89c71f9cc7eb7188a40b00..4c49d484b4ca810e5569b01cf85dbd2faff09fdd 100644
--- a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package spark.mllib.util
+package org.apache.spark.mllib.util
 
 import scala.collection.JavaConversions._
 import scala.util.Random
 
 import org.jblas.DoubleMatrix
 
-import spark.{RDD, SparkContext}
-import spark.mllib.regression.LabeledPoint
-import spark.mllib.regression.LabeledPoint
+import org.apache.spark.{RDD, SparkContext}
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.regression.LabeledPoint
 
 /**
  * Generate sample data used for Linear Data. This class generates
diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
similarity index 95%
rename from mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
index d6402f23e2d94dd5724593622e00b97c52162b72..f553298fc5c22b891c44cdc937299abed33353db 100644
--- a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.mllib.util
+package org.apache.spark.mllib.util
 
 import scala.util.Random
 
-import spark.{RDD, SparkContext}
-import spark.mllib.regression.LabeledPoint
+import org.apache.spark.{RDD, SparkContext}
+import org.apache.spark.mllib.regression.LabeledPoint
 
 /**
  * Generate test data for LogisticRegression. This class chooses positive labels
diff --git a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
similarity index 97%
rename from mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
index 88992cde0cf8ed81a488673b74467e90d535ebd7..7eb69ae81cbca7ac5947b6104e6782a6e8448a58 100644
--- a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.mllib.recommendation
+package org.apache.spark.mllib.recommendation
 
 import scala.util.Random
 
 import org.jblas.DoubleMatrix
 
-import spark.{RDD, SparkContext}
-import spark.mllib.util.MLUtils
+import org.apache.spark.{RDD, SparkContext}
+import org.apache.spark.mllib.util.MLUtils
 
 /**
 * Generate RDD(s) containing data for Matrix Factorization.
@@ -110,4 +110,4 @@ object MFDataGenerator{
     sc.stop()
   
   }
-}
\ No newline at end of file
+}
diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
similarity index 96%
rename from mllib/src/main/scala/spark/mllib/util/MLUtils.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
index a8e6ae9953d3b1f6d3bbbe6d83e67db5e2fc6f9c..0aeafbe23c751fd8500e4a0ee4485d55e3d934cf 100644
--- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.mllib.util
+package org.apache.spark.mllib.util
 
-import spark.{RDD, SparkContext}
-import spark.SparkContext._
+import org.apache.spark.{RDD, SparkContext}
+import org.apache.spark.SparkContext._
 
 import org.jblas.DoubleMatrix
-import spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.regression.LabeledPoint
 
 /**
  * Helper methods to load, save and pre-process data used in ML Lib.
diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
similarity index 91%
rename from mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
index eff456cad637d35a43bdc1c8a849c6f6cc117c6e..d3f191b05b19d3a2170ebbeaf219fb5d0afcceb7 100644
--- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
@@ -1,11 +1,11 @@
-package spark.mllib.util
+package org.apache.spark.mllib.util
 
 import scala.util.Random
 
-import spark.{RDD, SparkContext}
+import org.apache.spark.{RDD, SparkContext}
 
 import org.jblas.DoubleMatrix
-import spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.regression.LabeledPoint
 
 /**
  * Generate sample data used for SVM. This class generates uniform random values
diff --git a/mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java
similarity index 93%
rename from mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java
rename to mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java
index e0ebd45cd8d7ceb8326904674a076ad74421b340..e18e3bc6a86beda897e4b15b539468a51c775704 100644
--- a/mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.classification;
+package org.apache.spark.mllib.classification;
 
 import java.io.Serializable;
 import java.util.List;
@@ -25,10 +25,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
 
-import spark.mllib.regression.LabeledPoint;
+import org.apache.spark.mllib.regression.LabeledPoint;
 
 public class JavaLogisticRegressionSuite implements Serializable {
   private transient JavaSparkContext sc;
diff --git a/mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java
similarity index 93%
rename from mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java
rename to mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java
index 7881b3c38f01589fcee0e29a1f96947176a9edc0..117e5eaa8b78e42294738f9faca2b0c6c5cd36f4 100644
--- a/mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.classification;
+package org.apache.spark.mllib.classification;
 
 
 import java.io.Serializable;
@@ -26,10 +26,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
 
-import spark.mllib.regression.LabeledPoint;
+import org.apache.spark.mllib.regression.LabeledPoint;
 
 public class JavaSVMSuite implements Serializable {
   private transient JavaSparkContext sc;
diff --git a/mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
similarity index 96%
rename from mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java
rename to mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
index 3f2d82bfb4a13fcb9a1bcbf7135dd2cccc85e4b5..32d3934ac135a9ecb4aa2df50b03b88dcd3addbe 100644
--- a/mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.clustering;
+package org.apache.spark.mllib.clustering;
 
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -26,8 +26,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
 
 public class JavaKMeansSuite implements Serializable {
   private transient JavaSparkContext sc;
diff --git a/mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
similarity index 96%
rename from mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java
rename to mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
index 7993629a6d5447fe9a75de51563a85d2481638d8..3323f6cee2b910d7e6f4fcf6d531b443a98e8b89 100644
--- a/mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.recommendation;
+package org.apache.spark.mllib.recommendation;
 
 import java.io.Serializable;
 import java.util.List;
@@ -27,8 +27,8 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
 
 import org.jblas.DoubleMatrix;
 
diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java
similarity index 94%
rename from mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java
rename to mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java
index 5863140bafec8c6f5e70b3b219a08ba25abe90cb..f44b25cd44d19be342ac6ee0ddfcb597fe515ab1 100644
--- a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression;
+package org.apache.spark.mllib.regression;
 
 import java.io.Serializable;
 import java.util.List;
@@ -25,9 +25,9 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.mllib.util.LinearDataGenerator;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.mllib.util.LinearDataGenerator;
 
 public class JavaLassoSuite implements Serializable {
   private transient JavaSparkContext sc;
diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java
similarity index 94%
rename from mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java
rename to mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java
index 50716c786179158ee940807afd015cca0291da48..5a4410a632649ca99b22536adf198ffef9840827 100644
--- a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression;
+package org.apache.spark.mllib.regression;
 
 import java.io.Serializable;
 import java.util.List;
@@ -25,9 +25,9 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.mllib.util.LinearDataGenerator;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.mllib.util.LinearDataGenerator;
 
 public class JavaLinearRegressionSuite implements Serializable {
   private transient JavaSparkContext sc;
diff --git a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java
similarity index 95%
rename from mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java
rename to mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java
index 2c0aabad309cbd9330cab9badf742ce7b7e366ac..2fdd5fc8fdca6371b0a85351451a306bbd2fc459 100644
--- a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression;
+package org.apache.spark.mllib.regression;
 
 import java.io.Serializable;
 import java.util.List;
@@ -27,9 +27,9 @@ import org.junit.Test;
 
 import org.jblas.DoubleMatrix;
 
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaSparkContext;
-import spark.mllib.util.LinearDataGenerator;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.mllib.util.LinearDataGenerator;
 
 public class JavaRidgeRegressionSuite implements Serializable {
   private transient JavaSparkContext sc;
diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
similarity index 97%
rename from mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala
rename to mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
index bd87c528c35db3daf0f95d05024c0dbf07f74595..34c67294e9ac9942f747a06db712d52e71f5e42b 100644
--- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.classification
+package org.apache.spark.mllib.classification
 
 import scala.util.Random
 import scala.collection.JavaConversions._
@@ -24,8 +24,8 @@ import org.scalatest.BeforeAndAfterAll
 import org.scalatest.FunSuite
 import org.scalatest.matchers.ShouldMatchers
 
-import spark.SparkContext
-import spark.mllib.regression._
+import org.apache.spark.SparkContext
+import org.apache.spark.mllib.regression._
 
 object LogisticRegressionSuite {
 
diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala
similarity index 96%
rename from mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala
rename to mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala
index 894ae458ad03d60ed4b1fe5652e07f7ad2bf7398..6a957e3ddca719188677c4bd150d113a8ab98204 100644
--- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.classification
+package org.apache.spark.mllib.classification
 
 import scala.util.Random
 import scala.math.signum
@@ -24,11 +24,11 @@ import scala.collection.JavaConversions._
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.FunSuite
 
-import spark.SparkContext
-import spark.mllib.regression._
-
 import org.jblas.DoubleMatrix
 
+import org.apache.spark.{SparkException, SparkContext}
+import org.apache.spark.mllib.regression._
+
 object SVMSuite {
 
   def generateSVMInputAsList(
@@ -159,7 +159,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll {
       }
     }
 
-    intercept[spark.SparkException] {
+    intercept[SparkException] {
       val model = SVMWithSGD.train(testRDDInvalid, 100)
     }
 
diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
similarity index 98%
rename from mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala
rename to mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
index d5d95c86393d21c345a9f9b164dead08b7689c2e..94245f6027b3094bba9cae70e19d359bce136b78 100644
--- a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-package spark.mllib.clustering
+package org.apache.spark.mllib.clustering
 
 import scala.util.Random
 
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.FunSuite
 
-import spark.SparkContext
-import spark.SparkContext._
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
 
 import org.jblas._
 
diff --git a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
similarity index 97%
rename from mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala
rename to mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
index 15a60efda6ec785fa802621f56c6562a756fd6d6..347ef238f4042ceb80379e368b9586a7ccf18a78 100644
--- a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.recommendation
+package org.apache.spark.mllib.recommendation
 
 import scala.collection.JavaConversions._
 import scala.util.Random
@@ -23,8 +23,8 @@ import scala.util.Random
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.FunSuite
 
-import spark.SparkContext
-import spark.SparkContext._
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
 
 import org.jblas._
 
diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
similarity index 96%
rename from mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala
rename to mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
index 622dbbab7fe9eadff3f42499901ae43e8541aa08..db980c7bae64f946e64555c53f04969babed25c1 100644
--- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression
+package org.apache.spark.mllib.regression
 
 import scala.collection.JavaConversions._
 import scala.util.Random
@@ -23,8 +23,8 @@ import scala.util.Random
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.FunSuite
 
-import spark.SparkContext
-import spark.mllib.util.LinearDataGenerator
+import org.apache.spark.SparkContext
+import org.apache.spark.mllib.util.LinearDataGenerator
 
 
 class LassoSuite extends FunSuite with BeforeAndAfterAll {
diff --git a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
similarity index 93%
rename from mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala
rename to mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
index acc48a3283cdd2158b958f3af0e41d23b5499762..ef500c704c8a9949e85fb40885669b500a921f1b 100644
--- a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression
+package org.apache.spark.mllib.regression
 
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.FunSuite
 
-import spark.SparkContext
-import spark.SparkContext._
-import spark.mllib.util.LinearDataGenerator
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
+import org.apache.spark.mllib.util.LinearDataGenerator
 
 class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll {
   @transient private var sc: SparkContext = _
diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
similarity index 94%
rename from mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala
rename to mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
index c4820357062d77d26c4e9f0c8e84df4c8b159585..c18092d804fa3e17055238aeb250a99e459e2d59 100644
--- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.mllib.regression
+package org.apache.spark.mllib.regression
 
 import scala.collection.JavaConversions._
 import scala.util.Random
@@ -24,9 +24,9 @@ import org.jblas.DoubleMatrix
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.FunSuite
 
-import spark.SparkContext
-import spark.SparkContext._
-import spark.mllib.util.LinearDataGenerator
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
+import org.apache.spark.mllib.util.LinearDataGenerator
 
 class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll {
   @transient private var sc: SparkContext = _
diff --git a/pom.xml b/pom.xml
index e2fd54a9660aeb84919467a8bc3f96a1be6e509b..9230611eaea43e002b255ed97b77135356865a9c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -18,22 +18,22 @@
 
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-parent</artifactId>
   <version>0.8.0-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Spark Project Parent POM</name>
-  <url>http://spark-project.org/</url>
+  <url>http://spark.incubator.apache.org/</url>
   <licenses>
     <license>
-      <name>BSD License</name>
-      <url>https://github.com/mesos/spark/blob/master/LICENSE</url>
+      <name>Apache 2.0 License</name>
+      <url>http://www.apache.org/licenses/LICENSE-2.0.html</url>
       <distribution>repo</distribution>
     </license>
   </licenses>
   <scm>
-    <connection>scm:git:git@github.com:mesos/spark.git</connection>
-    <url>scm:git:git@github.com:mesos/spark.git</url>
+    <connection>scm:git:git@github.com:apache/incubator-spark.git</connection>
+    <url>scm:git:git@github.com:apache/incubator-spark.git</url>
   </scm>
   <developers>
     <developer>
@@ -46,7 +46,7 @@
     </developer>
   </developers>
   <issueManagement>
-    <system>github</system>
+    <system>JIRA</system>
     <url>https://spark-project.atlassian.net/browse/SPARK</url>
   </issueManagement>
 
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 2e2681267137ca5bd29df9469ba27dae869eb288..18e86d2cae74f061d040d0b47baca2b58eaf37c1 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -74,7 +74,7 @@ object SparkBuild extends Build {
     core, repl, examples, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef
 
   def sharedSettings = Defaults.defaultSettings ++ Seq(
-    organization := "org.spark-project",
+    organization := "org.apache.spark",
     version := "0.8.0-SNAPSHOT",
     scalaVersion := "2.9.3",
     scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"),
@@ -103,7 +103,7 @@ object SparkBuild extends Build {
     //useGpg in Global := true,
 
     pomExtra := (
-      <url>http://spark-project.org/</url>
+      <url>http://spark.incubator.apache.org/</url>
       <licenses>
         <license>
           <name>Apache 2.0 License</name>
@@ -112,8 +112,8 @@ object SparkBuild extends Build {
         </license>
       </licenses>
       <scm>
-        <connection>scm:git:git@github.com:mesos/spark.git</connection>
-        <url>scm:git:git@github.com:mesos/spark.git</url>
+        <connection>scm:git:git@github.com:apache/incubator-spark.git</connection>
+        <url>scm:git:git@github.com:apache/incubator-spark.git</url>
       </scm>
       <developers>
         <developer>
@@ -125,6 +125,10 @@ object SparkBuild extends Build {
           <organizationUrl>http://www.cs.berkeley.edu/</organizationUrl>
         </developer>
       </developers>
+      <issueManagement>
+        <system>JIRA</system>
+        <url>https://spark-project.atlassian.net/browse/SPARK</url>
+      </issueManagement>
     ),
 
 /*
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 2803ce90f3ed2629a5b1e7934ee65e55b94a8f5a..906e9221a1d0c44b11509cdac44073d7f830c436 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -114,9 +114,9 @@ class SparkContext(object):
             self.addPyFile(path)
 
         # Create a temporary directory inside spark.local.dir:
-        local_dir = self._jvm.spark.Utils.getLocalDir()
+        local_dir = self._jvm.org.apache.spark.Utils.getLocalDir()
         self._temp_dir = \
-            self._jvm.spark.Utils.createTempDir(local_dir).getAbsolutePath()
+            self._jvm.org.apache.spark.Utils.createTempDir(local_dir).getAbsolutePath()
 
     @property
     def defaultParallelism(self):
diff --git a/python/pyspark/files.py b/python/pyspark/files.py
index 89bcbcfe068557f8b8351225b5f331f00976d8be..57ee14eeb777667fa89518aa4d57c03f87edd0fb 100644
--- a/python/pyspark/files.py
+++ b/python/pyspark/files.py
@@ -52,4 +52,4 @@ class SparkFiles(object):
             return cls._root_directory
         else:
             # This will have to change if we support multiple SparkContexts:
-            return cls._sc._jvm.spark.SparkFiles.getRootDirectory()
+            return cls._sc._jvm.org.apache.spark.SparkFiles.getRootDirectory()
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 3ccf062c86c665c26c5b816e71dbf9d3fa4d5e65..26fbe0f08045d30e0f07028949f3a783e7551035 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -53,7 +53,7 @@ def launch_gateway():
     # Connect to the gateway
     gateway = JavaGateway(GatewayClient(port=port), auto_convert=False)
     # Import the classes used by PySpark
-    java_import(gateway.jvm, "spark.api.java.*")
-    java_import(gateway.jvm, "spark.api.python.*")
+    java_import(gateway.jvm, "org.apache.spark.api.java.*")
+    java_import(gateway.jvm, "org.apache.spark.api.python.*")
     java_import(gateway.jvm, "scala.Tuple2")
     return gateway
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
index 919e35f24041a52531562662a83745882971d2b3..6a1b09e8df7beb62499b190468d9b90338d23712 100644
--- a/repl-bin/pom.xml
+++ b/repl-bin/pom.xml
@@ -19,13 +19,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-repl-bin</artifactId>
   <packaging>pom</packaging>
   <name>Spark Project REPL binary packaging</name>
@@ -39,18 +39,18 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-core</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-bagel</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-repl</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
@@ -109,7 +109,7 @@
       <id>hadoop2-yarn</id>
       <dependencies>
         <dependency>
-          <groupId>org.spark-project</groupId>
+          <groupId>org.apache.spark</groupId>
           <artifactId>spark-yarn</artifactId>
           <version>${project.version}</version>
         </dependency>
diff --git a/repl/pom.xml b/repl/pom.xml
index f800664cff68d2d8190d4f501bf56ff9e445b7a8..f6276f18958581cc3ca92b789d290dff684e1c38 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -19,13 +19,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-repl</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project REPL</name>
@@ -38,18 +38,18 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-core</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-bagel</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-mllib</artifactId>
       <version>${project.version}</version>
       <scope>runtime</scope>
@@ -136,7 +136,7 @@
       <id>hadoop2-yarn</id>
       <dependencies>
         <dependency>
-          <groupId>org.spark-project</groupId>
+          <groupId>org.apache.spark</groupId>
           <artifactId>spark-yarn</artifactId>
           <version>${project.version}</version>
         </dependency>
diff --git a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
similarity index 99%
rename from repl/src/main/scala/spark/repl/ExecutorClassLoader.scala
rename to repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
index 274bc585db91eccaafb40988e37b6a18f1084d9f..3e171849e34941061ee63b01a8157edfe0bf4b25 100644
--- a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import java.io.{ByteArrayOutputStream, InputStream}
 import java.net.{URI, URL, URLClassLoader, URLEncoder}
diff --git a/repl/src/main/scala/spark/repl/Main.scala b/repl/src/main/scala/org/apache/spark/repl/Main.scala
similarity index 97%
rename from repl/src/main/scala/spark/repl/Main.scala
rename to repl/src/main/scala/org/apache/spark/repl/Main.scala
index d824d62fd10110ec7b9c5323d10ba8c603d34730..17e149f8abcc9ac80077ea8ef31bed4e205e1de5 100644
--- a/repl/src/main/scala/spark/repl/Main.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/Main.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import scala.collection.mutable.Set
 
diff --git a/repl/src/main/scala/spark/repl/SparkHelper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala
similarity index 100%
rename from repl/src/main/scala/spark/repl/SparkHelper.scala
rename to repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala
diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
similarity index 98%
rename from repl/src/main/scala/spark/repl/SparkILoop.scala
rename to repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
index 0cecbd71adc9f8f63d023cf5e0b21aaee0543fb9..193ccb48eeb98827bb8d4f6f6df62209f7c5010a 100644
--- a/repl/src/main/scala/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -3,7 +3,7 @@
  * @author Alexander Spoon
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import scala.tools.nsc._
 import scala.tools.nsc.interpreter._
@@ -22,8 +22,8 @@ import util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream }
 import interpreter._
 import io.{ File, Sources }
 
-import spark.Logging
-import spark.SparkContext
+import org.apache.spark.Logging
+import org.apache.spark.SparkContext
 
 /** The Scala interactive shell.  It provides a read-eval-print loop
  *  around the Interpreter class.
@@ -816,13 +816,13 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
   def initializeSpark() {
     intp.beQuietDuring {
       command("""
-        spark.repl.Main.interp.out.println("Creating SparkContext...");
-        spark.repl.Main.interp.out.flush();
-        @transient val sc = spark.repl.Main.interp.createSparkContext();
-        spark.repl.Main.interp.out.println("Spark context available as sc.");
-        spark.repl.Main.interp.out.flush();
+        org.apache.spark.repl.Main.interp.out.println("Creating SparkContext...");
+        org.apache.spark.repl.Main.interp.out.flush();
+        @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext();
+        org.apache.spark.repl.Main.interp.out.println("Spark context available as sc.");
+        org.apache.spark.repl.Main.interp.out.flush();
         """)
-      command("import spark.SparkContext._")
+      command("import org.apache.spark.SparkContext._")
     }
     echo("Type in expressions to have them evaluated.")
     echo("Type :help for more information.")
diff --git a/repl/src/main/scala/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
similarity index 99%
rename from repl/src/main/scala/spark/repl/SparkIMain.scala
rename to repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
index 43b6a6c9500302acac56155a44a5b4c658b62912..7e244e48a2ae45c62813344ce529e661ae736b95 100644
--- a/repl/src/main/scala/spark/repl/SparkIMain.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
@@ -3,7 +3,7 @@
  * @author  Martin Odersky
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import scala.tools.nsc._
 import scala.tools.nsc.interpreter._
@@ -27,9 +27,9 @@ import scala.util.control.Exception.{ ultimately }
 import scala.reflect.NameTransformer
 import SparkIMain._
 
-import spark.HttpServer
-import spark.Utils
-import spark.SparkEnv
+import org.apache.spark.HttpServer
+import org.apache.spark.Utils
+import org.apache.spark.SparkEnv
 
 /** An interpreter for Scala code.
  *  
@@ -883,7 +883,7 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
         val execution = lineManager.set(originalLine) {
           // MATEI: set the right SparkEnv for our SparkContext, because
           // this execution will happen in a separate thread
-          val sc = spark.repl.Main.interp.sparkContext
+          val sc = org.apache.spark.repl.Main.interp.sparkContext
           if (sc != null && sc.env != null)
             SparkEnv.set(sc.env)
           // Execute the line
diff --git a/repl/src/main/scala/spark/repl/SparkISettings.scala b/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala
similarity index 98%
rename from repl/src/main/scala/spark/repl/SparkISettings.scala
rename to repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala
index 8ebb01d14602171c3ab8c38eaf9df79ec629e0a8..605b7b259b54f82a4e2e2b00d18c531ec2eaf77c 100644
--- a/repl/src/main/scala/spark/repl/SparkISettings.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala
@@ -3,7 +3,7 @@
  * @author Alexander Spoon
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import scala.tools.nsc._
 import scala.tools.nsc.interpreter._
diff --git a/repl/src/main/scala/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala
similarity index 99%
rename from repl/src/main/scala/spark/repl/SparkImports.scala
rename to repl/src/main/scala/org/apache/spark/repl/SparkImports.scala
index 5caf5ca51a407d573abd728004b7aa123a3e94b8..41a1731d600011a1d88073517c3606226ead25c3 100644
--- a/repl/src/main/scala/spark/repl/SparkImports.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala
@@ -3,7 +3,7 @@
  * @author  Paul Phillips
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import scala.tools.nsc._
 import scala.tools.nsc.interpreter._
diff --git a/repl/src/main/scala/spark/repl/SparkJLineCompletion.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala
similarity index 99%
rename from repl/src/main/scala/spark/repl/SparkJLineCompletion.scala
rename to repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala
index 0069d8b2f49c74d565f85a3a7bad12362f5ff3ec..fdc172d7530ae15996b72eb117988caa0ac519a3 100644
--- a/repl/src/main/scala/spark/repl/SparkJLineCompletion.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala
@@ -3,7 +3,7 @@
  * @author Paul Phillips
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import scala.tools.nsc._
 import scala.tools.nsc.interpreter._
diff --git a/repl/src/main/scala/spark/repl/SparkJLineReader.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala
similarity index 98%
rename from repl/src/main/scala/spark/repl/SparkJLineReader.scala
rename to repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala
index ef6b6e092e04df8143d0a0d72ac489a32707bed5..d9e1de105c2161cf1ad2d6b7196c2035d74b12f4 100644
--- a/repl/src/main/scala/spark/repl/SparkJLineReader.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala
@@ -3,7 +3,7 @@
  * @author Stepan Koltsov
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import scala.tools.nsc._
 import scala.tools.nsc.interpreter._
diff --git a/repl/src/main/scala/spark/repl/SparkMemberHandlers.scala b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala
similarity index 99%
rename from repl/src/main/scala/spark/repl/SparkMemberHandlers.scala
rename to repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala
index 2980dfcd76834652b0a4e205253f99c93d8b6789..a3409bf66542caa094dfedca3af53fbc0cb608c0 100644
--- a/repl/src/main/scala/spark/repl/SparkMemberHandlers.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala
@@ -3,7 +3,7 @@
  * @author  Martin Odersky
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import scala.tools.nsc._
 import scala.tools.nsc.interpreter._
diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
similarity index 98%
rename from repl/src/test/scala/spark/repl/ReplSuite.scala
rename to repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
index 80ae605558015613eccee56d82eafe6ce59d0aa2..8f9b632c0eea67ace92595a389cbe6755902b161 100644
--- a/repl/src/test/scala/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.repl
+package org.apache.spark.repl
 
 import java.io._
 import java.net.URLClassLoader
@@ -41,10 +41,10 @@ class ReplSuite extends FunSuite {
       }
     }
     val interp = new SparkILoop(in, new PrintWriter(out), master)
-    spark.repl.Main.interp = interp
+    org.apache.spark.repl.Main.interp = interp
     val separator = System.getProperty("path.separator")
     interp.process(Array("-classpath", paths.mkString(separator)))
-    spark.repl.Main.interp = null
+    org.apache.spark.repl.Main.interp = null
     if (interp.sparkContext != null) {
       interp.sparkContext.stop()
     }
diff --git a/spark-executor b/spark-executor
index 63692bd46c939428ee598b43ea8a041cce808fd6..2c07c5484338fc9acddf4a4c74130566efba5176 100755
--- a/spark-executor
+++ b/spark-executor
@@ -19,4 +19,4 @@
 
 FWDIR="`dirname $0`"
 echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/spark-class spark.executor.MesosExecutorBackend
+exec $FWDIR/spark-class org.apache.spark.executor.MesosExecutorBackend
diff --git a/spark-shell b/spark-shell
index 4d379c5cfb9f3adbabe28d9d0db881143d24bd0d..9608bd3f30be79825a2c9915d39820e3787753a3 100755
--- a/spark-shell
+++ b/spark-shell
@@ -79,7 +79,7 @@ if [[ ! $? ]]; then
   saved_stty=""
 fi
 
-$FWDIR/spark-class $OPTIONS spark.repl.Main "$@"
+$FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
 
 # record the exit status lest it be overwritten:
 # then reenable echo and propagate the code.
diff --git a/spark-shell.cmd b/spark-shell.cmd
index ec65eabb7413fdb5ab102f1ee17c8ba81ad113c5..b9b4d4bfb2b760049d54bb5d8458de65e14484a5 100644
--- a/spark-shell.cmd
+++ b/spark-shell.cmd
@@ -19,4 +19,4 @@ rem
 
 set FWDIR=%~dp0
 set SPARK_LAUNCH_WITH_SCALA=1
-cmd /V /E /C %FWDIR%run2.cmd spark.repl.Main %*
+cmd /V /E /C %FWDIR%run2.cmd org.apache.spark.repl.Main %*
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 5c0582d6fb631d80c604cab85f789d19d549a2d4..c8946313ee9d0c514023c3d825a1224f90e4e76a 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -19,13 +19,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-streaming</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Streaming</name>
@@ -41,7 +41,7 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-core</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
similarity index 98%
rename from streaming/src/main/scala/spark/streaming/Checkpoint.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index 070d930b5eb8f5bf0783f42007bc4cfd99813625..2d8f07262426f00373a7afc4ea0064887b60ff2f 100644
--- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 import java.io._
 import java.util.concurrent.Executors
@@ -24,8 +24,8 @@ import java.util.concurrent.RejectedExecutionException
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.conf.Configuration
 
-import spark.Logging
-import spark.io.CompressionCodec
+import org.apache.spark.Logging
+import org.apache.spark.io.CompressionCodec
 
 
 private[streaming]
diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
similarity index 98%
rename from streaming/src/main/scala/spark/streaming/DStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
index 684d3abb564566d0d1f56397bc97305f448a05ff..362247cc381321c31e6558542747c5282a7cb789 100644
--- a/streaming/src/main/scala/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
@@ -15,14 +15,16 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
-import spark.streaming.dstream._
+import org.apache.spark.streaming.dstream._
 import StreamingContext._
+import org.apache.spark.util.MetadataCleaner
+
 //import Time._
 
-import spark.{RDD, Logging}
-import spark.storage.StorageLevel
+import org.apache.spark.{RDD, Logging}
+import org.apache.spark.storage.StorageLevel
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
@@ -34,7 +36,7 @@ import org.apache.hadoop.conf.Configuration
 
 /**
  * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
- * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]]
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]]
  * for more details on RDDs). DStreams can either be created from live data (such as, data from
  * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
  * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
@@ -42,7 +44,7 @@ import org.apache.hadoop.conf.Configuration
  * by a parent DStream.
  *
  * This class contains the basic operations available on all DStreams, such as `map`, `filter` and
- * `window`. In addition, [[spark.streaming.PairDStreamFunctions]] contains operations available
+ * `window`. In addition, [[org.apache.spark.streaming.PairDStreamFunctions]] contains operations available
  * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. These operations
  * are automatically available on any DStream of the right type (e.g., DStream[(Int, Int)] through
  * implicit conversions when `spark.streaming.StreamingContext._` is imported.
@@ -209,7 +211,7 @@ abstract class DStream[T: ClassManifest] (
         checkpointDuration + "). Please set it to higher than " + checkpointDuration + "."
     )
 
-    val metadataCleanerDelay = spark.util.MetadataCleaner.getDelaySeconds
+    val metadataCleanerDelay = MetadataCleaner.getDelaySeconds
     logInfo("metadataCleanupDelay = " + metadataCleanerDelay)
     assert(
       metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000,
diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
similarity index 98%
rename from streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
index 399ca1c63d8b7c1644d1e8cf74fdc884609d4707..58a0da28705411a8aa64ba4d7cf44b5438878cb6 100644
--- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.fs.FileSystem
 import org.apache.hadoop.conf.Configuration
 import collection.mutable.HashMap
-import spark.Logging
+import org.apache.spark.Logging
 
 
 
diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
similarity index 98%
rename from streaming/src/main/scala/spark/streaming/DStreamGraph.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
index c09a332d44cc5a8811ad9ac66d030787915c5fdf..b9a58fded67614d40d7a4363badb6ac30dc844fd 100644
--- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 import dstream.InputDStream
 import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
 import collection.mutable.ArrayBuffer
-import spark.Logging
+import org.apache.spark.Logging
 
 final private[streaming] class DStreamGraph extends Serializable with Logging {
   initLogging()
diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala
similarity index 86%
rename from streaming/src/main/scala/spark/streaming/Duration.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/Duration.scala
index 12a14e233d93779cb44075583c463c5606f3b543..290ad3781235d0fd1c49f63cbf14ac8bd9b0f9c8 100644
--- a/streaming/src/main/scala/spark/streaming/Duration.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
-import spark.Utils
+import org.apache.spark.Utils
 
 case class Duration (private val millis: Long) {
 
@@ -57,7 +57,7 @@ case class Duration (private val millis: Long) {
 }
 
 /**
- * Helper object that creates instance of [[spark.streaming.Duration]] representing
+ * Helper object that creates instance of [[org.apache.spark.streaming.Duration]] representing
  * a given number of milliseconds.
  */
 object Milliseconds {
@@ -65,7 +65,7 @@ object Milliseconds {
 }
 
 /**
- * Helper object that creates instance of [[spark.streaming.Duration]] representing
+ * Helper object that creates instance of [[org.apache.spark.streaming.Duration]] representing
  * a given number of seconds.
  */
 object Seconds {
@@ -73,7 +73,7 @@ object Seconds {
 }
 
 /**
- * Helper object that creates instance of [[spark.streaming.Duration]] representing
+ * Helper object that creates instance of [[org.apache.spark.streaming.Duration]] representing
  * a given number of minutes.
  */
 object Minutes {
diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala
similarity index 98%
rename from streaming/src/main/scala/spark/streaming/Interval.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/Interval.scala
index b30cd969e9fd0706842813c32a50652dc757a571..04c994c136932a19bce2a12e088444e5feacb280 100644
--- a/streaming/src/main/scala/spark/streaming/Interval.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 private[streaming]
 class Interval(val beginTime: Time, val endTime: Time) {
diff --git a/streaming/src/main/scala/spark/streaming/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/Job.scala
similarity index 97%
rename from streaming/src/main/scala/spark/streaming/Job.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/Job.scala
index ceb3f92b65b51c707830ac2700d656e12effa5a3..2128b7c7a64c27c98a8d88db6d27f801b8cf606e 100644
--- a/streaming/src/main/scala/spark/streaming/Job.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Job.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 import java.util.concurrent.atomic.AtomicLong
 
diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala
similarity index 96%
rename from streaming/src/main/scala/spark/streaming/JobManager.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala
index a31230689f95f3c26b0657b77e692141aa50cc9c..5233129506f9e8b2cdd4b0a208d8d01972564d0d 100644
--- a/streaming/src/main/scala/spark/streaming/JobManager.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
-import spark.Logging 
-import spark.SparkEnv
+import org.apache.spark.Logging
+import org.apache.spark.SparkEnv
 import java.util.concurrent.Executors
 import collection.mutable.HashMap
 import collection.mutable.ArrayBuffer
diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
similarity index 95%
rename from streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
index d4cf2e568ca34062596edfe46c5be83a037591c8..aae79a4e6fefd61d199fe46c95bf73be4a2a8c4b 100644
--- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
-import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver}
-import spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError}
-import spark.Logging
-import spark.SparkEnv
-import spark.SparkContext._
+import org.apache.spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver}
+import org.apache.spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError}
+import org.apache.spark.Logging
+import org.apache.spark.SparkEnv
+import org.apache.spark.SparkContext._
 
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.Queue
diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
similarity index 96%
rename from streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
index 47bf07bee1318c24ba20ceec58559a6d4be74021..d8a7381e87d616c147de495b556d09c351f71eea 100644
--- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
-import spark.streaming.StreamingContext._
-import spark.streaming.dstream.{ReducedWindowedDStream, StateDStream}
-import spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream}
-import spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream}
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.dstream.{ReducedWindowedDStream, StateDStream}
+import org.apache.spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream}
+import org.apache.spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream}
 
-import spark.{Manifests, RDD, Partitioner, HashPartitioner}
-import spark.SparkContext._
-import spark.storage.StorageLevel
+import org.apache.spark.{Manifests, RDD, Partitioner, HashPartitioner}
+import org.apache.spark.SparkContext._
+import org.apache.spark.storage.StorageLevel
 
 import scala.collection.mutable.ArrayBuffer
 
@@ -60,7 +60,7 @@ extends Serializable {
   }
 
   /**
-   * Return a new DStream by applying `groupByKey` on each RDD. The supplied [[spark.Partitioner]]
+   * Return a new DStream by applying `groupByKey` on each RDD. The supplied [[org.apache.spark.Partitioner]]
    * is used to control the partitioning of each RDD.
    */
   def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = {
@@ -91,7 +91,7 @@ extends Serializable {
 
   /**
    * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are
-   * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the
+   * merged using the supplied reduce function. [[org.apache.spark.Partitioner]] is used to control the
    * partitioning of each RDD.
    */
   def reduceByKey(reduceFunc: (V, V) => V, partitioner: Partitioner): DStream[(K, V)] = {
@@ -101,7 +101,7 @@ extends Serializable {
 
   /**
    * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the
-   * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more
+   * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more
    * information.
    */
   def combineByKey[C: ClassManifest](
@@ -360,7 +360,7 @@ extends Serializable {
   /**
    * Create a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of the key.
-   * [[spark.Partitioner]] is used to control the partitioning of each RDD.
+   * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
    * @param updateFunc State update function. If `this` function returns None, then
    *                   corresponding state key-value pair will be eliminated.
    * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
@@ -379,7 +379,7 @@ extends Serializable {
   /**
    * Return a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of each key.
-   * [[spark.Paxrtitioner]] is used to control the partitioning of each RDD.
+   * [[org.apache.spark.Paxrtitioner]] is used to control the partitioning of each RDD.
    * @param updateFunc State update function. If `this` function returns None, then
    *                   corresponding state key-value pair will be eliminated. Note, that
    *                   this function may generate a different a tuple with a different key
diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala
similarity index 95%
rename from streaming/src/main/scala/spark/streaming/Scheduler.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala
index 252cc2a30327c2945f6cad68cfa3966e953fabe6..ed892e33e6c3467f9d9aeef923dc2199a711b89b 100644
--- a/streaming/src/main/scala/spark/streaming/Scheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 import util.{ManualClock, RecurringTimer, Clock}
-import spark.SparkEnv
-import spark.Logging
+import org.apache.spark.SparkEnv
+import org.apache.spark.Logging
 
 private[streaming]
 class Scheduler(ssc: StreamingContext) extends Logging {
@@ -34,7 +34,8 @@ class Scheduler(ssc: StreamingContext) extends Logging {
     null
   }
 
-  val clockClass = System.getProperty("spark.streaming.clock", "spark.streaming.util.SystemClock")
+  val clockClass = System.getProperty(
+    "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
   val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
   val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
     longTime => generateJobs(new Time(longTime)))
diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
similarity index 97%
rename from streaming/src/main/scala/spark/streaming/StreamingContext.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 62c95b573aaa3d04f75bb966680a58f43d02b9e0..3852ac2daba5617c3f461f7cb39f866f1039e102 100644
--- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -15,21 +15,21 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 import akka.actor.Props
 import akka.actor.SupervisorStrategy
 import akka.zeromq.Subscribe
 
-import spark.streaming.dstream._
+import org.apache.spark.streaming.dstream._
 
-import spark._
-import spark.streaming.receivers.ActorReceiver
-import spark.streaming.receivers.ReceiverSupervisorStrategy
-import spark.streaming.receivers.ZeroMQReceiver
-import spark.storage.StorageLevel
-import spark.util.MetadataCleaner
-import spark.streaming.receivers.ActorReceiver
+import org.apache.spark._
+import org.apache.spark.streaming.receivers.ActorReceiver
+import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy
+import org.apache.spark.streaming.receivers.ZeroMQReceiver
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.MetadataCleaner
+import org.apache.spark.streaming.receivers.ActorReceiver
 
 import scala.collection.mutable.Queue
 import scala.collection.Map
diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala
similarity index 98%
rename from streaming/src/main/scala/spark/streaming/Time.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/Time.scala
index ad5eab9dd2aad52d5dca5a5f6bcbb7defdd5609f..2678334f53844f2c388d83e393ff305224071c66 100644
--- a/streaming/src/main/scala/spark/streaming/Time.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 /**
  * This is a simple class that represents an absolute instant of time.
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
similarity index 90%
rename from streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
index 7dcb1d713d922c70e58402c10ddf9f816e93c5d4..f8c8d8ece1e28cc3ae0efdf9ef95ce7e0d731867 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package spark.streaming.api.java
+package org.apache.spark.streaming.api.java
 
-import spark.streaming.{Duration, Time, DStream}
-import spark.api.java.function.{Function => JFunction}
-import spark.api.java.JavaRDD
-import spark.storage.StorageLevel
-import spark.RDD
+import org.apache.spark.streaming.{Duration, Time, DStream}
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.api.java.JavaRDD
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.RDD
 
 /**
  * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
- * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]]
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]]
  * for more details on RDDs). DStreams can either be created from live data (such as, data from
  * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
  * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
@@ -33,7 +33,7 @@ import spark.RDD
  * by a parent DStream.
  *
  * This class contains the basic operations available on all DStreams, such as `map`, `filter` and
- * `window`. In addition, [[spark.streaming.api.java.JavaPairDStream]] contains operations available
+ * `window`. In addition, [[org.apache.spark.streaming.api.java.JavaPairDStream]] contains operations available
  * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`.
  *
  * DStreams internally is characterized by a few basic properties:
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
similarity index 97%
rename from streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
index 3ab5c1fddeb297192f2be6afaa3f8705927743d0..2e6fe9a9c4fa88187afb174904d6ffc534afb00c 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
@@ -15,18 +15,18 @@
  * limitations under the License.
  */
 
-package spark.streaming.api.java
+package org.apache.spark.streaming.api.java
 
 import java.util.{List => JList}
 import java.lang.{Long => JLong}
 
 import scala.collection.JavaConversions._
 
-import spark.streaming._
-import spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD}
-import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
+import org.apache.spark.streaming._
+import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD}
+import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
 import java.util
-import spark.RDD
+import org.apache.spark.RDD
 import JavaDStream._
 
 trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
similarity index 97%
rename from streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
index ea08fb38267941807a524d3f38330798bfc0b4ff..c203dccd171d9eeca1718fe937258c96ec74e952 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
@@ -15,24 +15,24 @@
  * limitations under the License.
  */
 
-package spark.streaming.api.java
+package org.apache.spark.streaming.api.java
 
 import java.util.{List => JList}
 import java.lang.{Long => JLong}
 
 import scala.collection.JavaConversions._
 
-import spark.streaming._
-import spark.streaming.StreamingContext._
-import spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
-import spark.{RDD, Partitioner}
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
+import org.apache.spark.{RDD, Partitioner}
 import org.apache.hadoop.mapred.{JobConf, OutputFormat}
 import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
 import org.apache.hadoop.conf.Configuration
-import spark.api.java.{JavaUtils, JavaRDD, JavaPairRDD}
-import spark.storage.StorageLevel
+import org.apache.spark.api.java.{JavaUtils, JavaRDD, JavaPairRDD}
+import org.apache.spark.storage.StorageLevel
 import com.google.common.base.Optional
-import spark.RDD
+import org.apache.spark.RDD
 
 class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
     implicit val kManifiest: ClassManifest[K],
@@ -114,7 +114,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   /**
    * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream.
    * Therefore, the values for each key in `this` DStream's RDDs are grouped into a
-   * single sequence to generate the RDDs of the new DStream. [[spark.Partitioner]]
+   * single sequence to generate the RDDs of the new DStream. [[org.apache.spark.Partitioner]]
    * is used to control the partitioning of each RDD.
    */
   def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] =
@@ -138,7 +138,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
 
   /**
    * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are
-   * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the
+   * merged using the supplied reduce function. [[org.apache.spark.Partitioner]] is used to control the
    * partitioning of each RDD.
    */
   def reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner): JavaPairDStream[K, V] = {
@@ -147,7 +147,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
 
   /**
    * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
-   * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more
+   * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more
    * information.
    */
   def combineByKey[C](createCombiner: JFunction[V, C],
@@ -445,7 +445,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   /**
    * Create a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of the key.
-   * [[spark.Partitioner]] is used to control the partitioning of each RDD.
+   * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
    * @param updateFunc State update function. If `this` function returns None, then
    *                   corresponding state key-value pair will be eliminated.
    * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
similarity index 97%
rename from streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index b7720ad0ea1b527dbdc1d99afdf90f0eb46becc5..f10beb1db399160225db8600c8d4970e246fec4e 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.streaming.api.java
+package org.apache.spark.streaming.api.java
 
-import spark.streaming._
+import org.apache.spark.streaming._
 import receivers.{ActorReceiver, ReceiverSupervisorStrategy}
-import spark.streaming.dstream._
-import spark.storage.StorageLevel
-import spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
-import spark.api.java.{JavaSparkContext, JavaRDD}
+import org.apache.spark.streaming.dstream._
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
+import org.apache.spark.api.java.{JavaSparkContext, JavaRDD}
 import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
 import twitter4j.Status
 import akka.actor.Props
@@ -33,6 +33,7 @@ import java.lang.{Long => JLong, Integer => JInt}
 import java.io.InputStream
 import java.util.{Map => JMap}
 import twitter4j.auth.Authorization
+import org.apache.spark.RDD
 
 /**
  * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -537,7 +538,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   def queueStream[T](queue: java.util.Queue[JavaRDD[T]]): JavaDStream[T] = {
     implicit val cm: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
-    val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]]
+    val sQueue = new scala.collection.mutable.Queue[RDD[T]]
     sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
     ssc.queueStream(sQueue)
   }
@@ -554,7 +555,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
   def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = {
     implicit val cm: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
-    val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]]
+    val sQueue = new scala.collection.mutable.Queue[RDD[T]]
     sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
     ssc.queueStream(sQueue, oneAtATime)
   }
@@ -575,7 +576,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
       defaultRDD: JavaRDD[T]): JavaDStream[T] = {
     implicit val cm: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
-    val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]]
+    val sQueue = new scala.collection.mutable.Queue[RDD[T]]
     sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
     ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd)
   }
diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala
similarity index 90%
rename from streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala
index 99553d295d55b168694947e5a49f4ca02dc9d58a..4a9d82211fcaa018b312b08cb83cd73590da1422 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.{RDD, Partitioner}
-import spark.rdd.CoGroupedRDD
-import spark.streaming.{Time, DStream, Duration}
+import org.apache.spark.{RDD, Partitioner}
+import org.apache.spark.rdd.CoGroupedRDD
+import org.apache.spark.streaming.{Time, DStream, Duration}
 
 private[streaming]
 class CoGroupedDStream[K : ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala
similarity index 89%
rename from streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala
index 095137092a96735ab91ab7283cf6c4b1e0298c6f..35cc4cb396b842f3dce656f9311d0b1f649845b0 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.RDD
-import spark.streaming.{Time, StreamingContext}
+import org.apache.spark.RDD
+import org.apache.spark.streaming.{Time, StreamingContext}
 
 /**
  * An input stream that always returns the same RDD on each timestep. Useful for testing.
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
similarity index 97%
rename from streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
index de0536125de0e43412ca2af8f668661d07c7f4f7..1c265ed972d85a3cfe7ff7ffa53ed98ca08c0d32 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.RDD
-import spark.rdd.UnionRDD
-import spark.streaming.{DStreamCheckpointData, StreamingContext, Time}
+import org.apache.spark.RDD
+import org.apache.spark.rdd.UnionRDD
+import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time}
 
 import org.apache.hadoop.fs.{FileSystem, Path, PathFilter}
 import org.apache.hadoop.conf.Configuration
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala
similarity index 90%
rename from streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala
index 9d8c5c3175b9af7090b59026cc4c15d13d622272..3166c68760816223751af1cb0cf4e3e02ceaf24b 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Duration, DStream, Time}
-import spark.RDD
+import org.apache.spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.RDD
 
 private[streaming]
 class FilteredDStream[T: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala
similarity index 88%
rename from streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala
index 78d7117f0f71c9e5400f71ec3c33e761bbb4ea67..21950ad6ac9967ed0462808864fc8bbf305f0830 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Duration, DStream, Time}
-import spark.RDD
-import spark.SparkContext._
+import org.apache.spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.RDD
+import org.apache.spark.SparkContext._
 
 private[streaming]
 class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala
similarity index 90%
rename from streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala
index d13bebb10f63f032fedf0f36a8079e3866b85f02..8377cfe60cd78c617ef242748c827d5915faa935 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Duration, DStream, Time}
-import spark.RDD
+import org.apache.spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.RDD
 
 private[streaming]
 class FlatMappedDStream[T: ClassManifest, U: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
similarity index 96%
rename from streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
index 4906f503c29c65f520cb8850239a6d22d46d6e31..3fb443143cd2be0ff64cac62ec3b6f772073491c 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.StreamingContext
+import org.apache.spark.streaming.StreamingContext
 
-import spark.Utils
-import spark.storage.StorageLevel
+import org.apache.spark.Utils
+import org.apache.spark.storage.StorageLevel
 
 import org.apache.flume.source.avro.AvroSourceProtocol
 import org.apache.flume.source.avro.AvroFlumeEvent
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala
similarity index 91%
rename from streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala
index 7df537eb560e27d6c79669d38d7aa61f39fd5a3a..c1f95650c8df70926d01b14fa698b9ac43f77d5a 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.RDD
-import spark.streaming.{Duration, DStream, Job, Time}
+import org.apache.spark.RDD
+import org.apache.spark.streaming.{Duration, DStream, Job, Time}
 
 private[streaming]
 class ForEachDStream[T: ClassManifest] (
diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala
similarity index 89%
rename from streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala
index 06fda6fe8e40f8f3ad6081ae5afe8f90930fb759..1e4c7e7fde8eafb8e21df79f8f7e482de9083c92 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Duration, DStream, Time}
-import spark.RDD
+import org.apache.spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.RDD
 
 private[streaming]
 class GlommedDStream[T: ClassManifest](parent: DStream[T])
diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
similarity index 95%
rename from streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
index 4dbdec459d025de6a7d45f32e4e274de08f887eb..674b27118caef7fb07cbe6c3c3a2faa90d78051f 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Time, Duration, StreamingContext, DStream}
+import org.apache.spark.streaming.{Time, Duration, StreamingContext, DStream}
 
 /**
  * This is the abstract base class for all input streams. This class provides to methods
diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
similarity index 96%
rename from streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
index 6ee588af15451fd7df12b957bc0d583fc973f9ab..51e913675d24acbfd04534e6d94ed2edc2cdc4af 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.Logging
-import spark.storage.StorageLevel
-import spark.streaming.{Time, DStreamCheckpointData, StreamingContext}
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{Time, DStreamCheckpointData, StreamingContext}
 
 import java.util.Properties
 import java.util.concurrent.Executors
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala
similarity index 90%
rename from streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala
index af41a1b9acdf1b5402164b27c22ae0951035516b..1d79d707bb40a1cae9436637dd34db1938566590 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Duration, DStream, Time}
-import spark.RDD
+import org.apache.spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.RDD
 
 private[streaming]
 class MapPartitionedDStream[T: ClassManifest, U: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala
similarity index 87%
rename from streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala
index 8d8a6161c6750e81dc994be08026c4cceeea469c..312e0c0567da6798b2650f2ae032a42f541a2406 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Duration, DStream, Time}
-import spark.RDD
-import spark.SparkContext._
+import org.apache.spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.RDD
+import org.apache.spark.SparkContext._
 
 private[streaming]
 class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala
similarity index 90%
rename from streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala
index 3fda84a38a189b5a289305c7c51da60e42571188..af688dde5f3da76d977201f5e21b49f88b94d28c 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Duration, DStream, Time}
-import spark.RDD
+import org.apache.spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.RDD
 
 private[streaming]
 class MappedDStream[T: ClassManifest, U: ClassManifest] (
diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
similarity index 93%
rename from streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
index 1db0a69a2f109869a6a70b0d474565ef6fb66a50..3d68da36a27ac7b5ac610c9cadc883ca15051caf 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver}
+import org.apache.spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver}
 
-import spark.{Logging, SparkEnv, RDD}
-import spark.rdd.BlockRDD
-import spark.storage.StorageLevel
+import org.apache.spark.{Logging, SparkEnv, RDD}
+import org.apache.spark.rdd.BlockRDD
+import org.apache.spark.storage.StorageLevel
 
 import scala.collection.mutable.ArrayBuffer
 
@@ -31,14 +31,14 @@ import akka.actor.{Props, Actor}
 import akka.pattern.ask
 import akka.dispatch.Await
 import akka.util.duration._
-import spark.streaming.util.{RecurringTimer, SystemClock}
+import org.apache.spark.streaming.util.{RecurringTimer, SystemClock}
 import java.util.concurrent.ArrayBlockingQueue
 
 /**
  * Abstract class for defining any InputDStream that has to start a receiver on worker
  * nodes to receive external data. Specific implementations of NetworkInputDStream must
  * define the getReceiver() function that gets the receiver object of type
- * [[spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive
+ * [[org.apache.spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive
  * data.
  * @param ssc_ Streaming context that will execute this input stream
  * @tparam T Class type of the object of this stream
@@ -83,7 +83,7 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe
 
 /**
  * Abstract class of a receiver that can be run on worker nodes to receive external data. See
- * [[spark.streaming.dstream.NetworkInputDStream]] for an explanation.
+ * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] for an explanation.
  */
 abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Logging {
 
@@ -202,7 +202,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log
   }
 
   /**
-   * Batches objects created by a [[spark.streaming.dstream.NetworkReceiver]] and puts them into
+   * Batches objects created by a [[org.apache.spark.streaming.dstream.NetworkReceiver]] and puts them into
    * appropriately named blocks at regular intervals. This class starts two threads,
    * one to periodically start a new batch and prepare the previous batch of as a block,
    * the other to push the blocks into the block manager.
diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala
similarity index 91%
rename from streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala
index 33f7cd063f3d1da6a81bc7329b08b37b7ea48c9e..15782f5c119054555d9b9d0548e49bd0e4d7c3d3 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.StreamingContext
+import org.apache.spark.streaming.StreamingContext
 
 private[streaming]
 class PluggableInputDStream[T: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala
similarity index 90%
rename from streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala
index b269061b7370f63f7e2b58e8df62e542b1106220..b43ecaeebe0d17e6119b792801cd2f4678cf2bc6 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.RDD
-import spark.rdd.UnionRDD
+import org.apache.spark.RDD
+import org.apache.spark.rdd.UnionRDD
 
 import scala.collection.mutable.Queue
 import scala.collection.mutable.ArrayBuffer
-import spark.streaming.{Time, StreamingContext}
+import org.apache.spark.streaming.{Time, StreamingContext}
 
 private[streaming]
 class QueueInputDStream[T: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
similarity index 95%
rename from streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
index 236f74f575e9d7c284b0d5b8a1637023af6bed82..c91f12ecd7afd8f74857a1c4731718c0fca517da 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.Logging
-import spark.storage.StorageLevel
-import spark.streaming.StreamingContext
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
 
 import java.net.InetSocketAddress
 import java.nio.ByteBuffer
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
similarity index 94%
rename from streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
index 96260501ab7d25fc0d05000832c4d200bd043fb0..b6c672f899e4208192e64c708a08297cb6d95353 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
@@ -15,18 +15,18 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.StreamingContext._
+import org.apache.spark.streaming.StreamingContext._
 
-import spark.RDD
-import spark.rdd.{CoGroupedRDD, MapPartitionsRDD}
-import spark.Partitioner
-import spark.SparkContext._
-import spark.storage.StorageLevel
+import org.apache.spark.RDD
+import org.apache.spark.rdd.{CoGroupedRDD, MapPartitionsRDD}
+import org.apache.spark.Partitioner
+import org.apache.spark.SparkContext._
+import org.apache.spark.storage.StorageLevel
 
 import scala.collection.mutable.ArrayBuffer
-import spark.streaming.{Duration, Interval, Time, DStream}
+import org.apache.spark.streaming.{Duration, Interval, Time, DStream}
 
 private[streaming]
 class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
similarity index 88%
rename from streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
index 83b57b27f77f36795f62397a2e98ebd70708f442..3a0bd2acd714109a41c933bac8972e8a550b46ce 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.{RDD, Partitioner}
-import spark.SparkContext._
-import spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.{RDD, Partitioner}
+import org.apache.spark.SparkContext._
+import org.apache.spark.streaming.{Duration, DStream, Time}
 
 private[streaming]
 class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
similarity index 93%
rename from streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
index 5877b10e0e16ae0160dd5cbf3665a8a364b40d6f..e2539c73961380769d842574eaf31a2011662914 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.StreamingContext
-import spark.storage.StorageLevel
-import spark.util.NextIterator
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.NextIterator
 
 import java.io._
 import java.net.Socket
diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala
similarity index 94%
rename from streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala
index 4b46613d5e102d1c39b872594295e44e9c4933ce..c1c9f808f0883aa23b5f69904ff6d55f764bd1cf 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.RDD
-import spark.Partitioner
-import spark.SparkContext._
-import spark.storage.StorageLevel
-import spark.streaming.{Duration, Time, DStream}
+import org.apache.spark.RDD
+import org.apache.spark.Partitioner
+import org.apache.spark.SparkContext._
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{Duration, Time, DStream}
 
 private[streaming]
 class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
similarity index 90%
rename from streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
index e7fbc5bbcfa8e148655810113976dc8aa5b935c0..edba2032b45da231b6d5eafa35f62a790e5accac 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.RDD
-import spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.RDD
+import org.apache.spark.streaming.{Duration, DStream, Time}
 
 private[streaming]
 class TransformedDStream[T: ClassManifest, U: ClassManifest] (
diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala
similarity index 97%
rename from streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala
index f09a8b9f90757b44e6c91aa298c499dfaae6fffe..387e15b0e6d809f94fcbe94f3ac00ddd8026ebe8 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark._
-import spark.streaming._
+import org.apache.spark._
+import org.apache.spark.streaming._
 import storage.StorageLevel
 import twitter4j._
 import twitter4j.auth.Authorization
diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
similarity index 91%
rename from streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
index 3eaa9a7e7f4c6c469aa6f595bb29f9302a0df72b..97eab97b2fa33eb06d2f79213ea9c0ed3a089e23 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.streaming.{Duration, DStream, Time}
-import spark.RDD
+import org.apache.spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.RDD
 import collection.mutable.ArrayBuffer
-import spark.rdd.UnionRDD
+import org.apache.spark.rdd.UnionRDD
 
 private[streaming]
 class UnionDStream[T: ClassManifest](parents: Array[DStream[T]])
diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
similarity index 89%
rename from streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
index fd24d61730d4fa82db234f8de3ae056025aadf3d..dbbea39e812379986496f9dc532dd799fed1bce4 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.streaming.dstream
+package org.apache.spark.streaming.dstream
 
-import spark.RDD
-import spark.rdd.UnionRDD
-import spark.storage.StorageLevel
-import spark.streaming.{Duration, Interval, Time, DStream}
+import org.apache.spark.RDD
+import org.apache.spark.rdd.UnionRDD
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{Duration, Interval, Time, DStream}
 
 private[streaming]
 class WindowedDStream[T: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
similarity index 97%
rename from streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
index abeeff11b93b52b6d5fdc185a342eb7020687a9d..4b5d8c467e64fc1332a131fe385f0798e6262a99 100644
--- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.streaming.receivers
+package org.apache.spark.streaming.receivers
 
 import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy }
 import akka.actor.{ actorRef2Scala, ActorRef }
 import akka.actor.{ PossiblyHarmful, OneForOneStrategy }
 
-import spark.storage.StorageLevel
-import spark.streaming.dstream.NetworkReceiver
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.dstream.NetworkReceiver
 
 import java.util.concurrent.atomic.AtomicInteger
 
diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
similarity index 95%
rename from streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
index 22d554e7e4944312195377fa9ca83aa1e0a0e108..043bb8c8bf7959f23c07e6c5d753e4120e2d73d9 100644
--- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.streaming.receivers
+package org.apache.spark.streaming.receivers
 
 import akka.actor.Actor
 import akka.zeromq._
 
-import spark.Logging
+import org.apache.spark.Logging
 
 /**
  * A receiver to subscribe to ZeroMQ stream.
diff --git a/streaming/src/main/scala/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala
similarity index 98%
rename from streaming/src/main/scala/spark/streaming/util/Clock.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala
index d9ac722df5caebd7c9adbce555941ed943b258fc..f67bb2f6ac51a5bf90fb50ba10ac296ed6e7285f 100644
--- a/streaming/src/main/scala/spark/streaming/util/Clock.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming.util
+package org.apache.spark.streaming.util
 
 private[streaming]
 trait Clock {
diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
similarity index 98%
rename from streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
index 8ce5d8daf52dddc8ea15d48475039fdeed12aa8b..50d72298e48fc434fd994544c2506b179a2ebcd8 100644
--- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package spark.streaming.util
+package org.apache.spark.streaming.util
 
-import spark.{Logging, RDD}
-import spark.streaming._
-import spark.streaming.dstream.ForEachDStream
+import org.apache.spark.{Logging, RDD}
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.dstream.ForEachDStream
 import StreamingContext._
 
 import scala.util.Random
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
similarity index 96%
rename from streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
index bf04120293e17b44967cdc4ea5f4a3cc3d4db577..4e6ce6eabd7ba2fd3753b9f1b3ea72a17e6616a9 100644
--- a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming.util
+package org.apache.spark.streaming.util
 
-import spark.SparkContext
-import spark.SparkContext._
+import org.apache.spark.SparkContext
+import org.apache.spark.SparkContext._
 import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
 import scala.collection.JavaConversions.mapAsScalaMap
 
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
similarity index 94%
rename from streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
index 5cc6ad9dee30025e7dbb29b947c0eb0601835c9e..249f6a22aeb8a95ff746dceeebea4b1f8983fa50 100644
--- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.streaming.util
+package org.apache.spark.streaming.util
 
 import java.nio.ByteBuffer
-import spark.util.{RateLimitedOutputStream, IntParam}
+import org.apache.spark.util.{RateLimitedOutputStream, IntParam}
 import java.net.ServerSocket
-import spark.{Logging, KryoSerializer}
+import org.apache.spark.{Logging, KryoSerializer}
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
 import scala.io.Source
 import java.io.IOException
diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala
similarity index 98%
rename from streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
rename to streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala
index 7ecc44236d4168d8375061a780a74b6878b07b34..d644240405caa478f6b838473a8d7f7475615942 100644
--- a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming.util
+package org.apache.spark.streaming.util
 
 private[streaming]
 class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) {
diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
similarity index 98%
rename from streaming/src/test/java/spark/streaming/JavaAPISuite.java
rename to streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index 3b93790baa8ecb07c22f24692a5d4ba55e0b80d4..c0d729ff8767373549991e872906a2b63d5fd05d 100644
--- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming;
+package org.apache.spark.streaming;
 
 import com.google.common.base.Optional;
 import com.google.common.collect.Lists;
@@ -28,20 +28,20 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import scala.Tuple2;
-import spark.HashPartitioner;
-import spark.api.java.JavaPairRDD;
-import spark.api.java.JavaRDD;
-import spark.api.java.JavaRDDLike;
-import spark.api.java.JavaPairRDD;
-import spark.api.java.JavaSparkContext;
-import spark.api.java.function.*;
-import spark.storage.StorageLevel;
-import spark.streaming.api.java.JavaDStream;
-import spark.streaming.api.java.JavaPairDStream;
-import spark.streaming.api.java.JavaStreamingContext;
-import spark.streaming.JavaTestUtils;
-import spark.streaming.JavaCheckpointTestUtils;
-import spark.streaming.InputStreamsSuite;
+import org.apache.spark.HashPartitioner;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaRDDLike;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.*;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.streaming.JavaTestUtils;
+import org.apache.spark.streaming.JavaCheckpointTestUtils;
+import org.apache.spark.streaming.InputStreamsSuite;
 
 import java.io.*;
 import java.util.*;
@@ -59,7 +59,7 @@ public class JavaAPISuite implements Serializable {
 
   @Before
   public void setUp() {
-      System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock");
+      System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
       ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
     ssc.checkpoint("checkpoint");
   }
diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
similarity index 84%
rename from streaming/src/test/java/spark/streaming/JavaTestUtils.scala
rename to streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
index f9d25db8da669d3052509a460192d0dc68da6c9a..8a6604904de57d39626eac6ec5f2b19405dad833 100644
--- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
@@ -15,20 +15,21 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
 import java.util.{List => JList}
-import spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext}
-import spark.streaming._
+import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext}
+import org.apache.spark.streaming._
 import java.util.ArrayList
 import collection.JavaConversions._
+import org.apache.spark.api.java.JavaRDDLike
 
 /** Exposes streaming test functionality in a Java-friendly way. */
 trait JavaTestBase extends TestSuiteBase {
 
   /**
-   * Create a [[spark.streaming.TestInputStream]] and attach it to the supplied context.
+   * Create a [[org.apache.spark.streaming.TestInputStream]] and attach it to the supplied context.
    * The stream will be derived from the supplied lists of Java objects.
    **/
   def attachTestInputStream[T](
@@ -46,11 +47,11 @@ trait JavaTestBase extends TestSuiteBase {
 
   /**
    * Attach a provided stream to it's associated StreamingContext as a
-   * [[spark.streaming.TestOutputStream]].
+   * [[org.apache.spark.streaming.TestOutputStream]].
    **/
-  def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T, This, R],
-      R <: spark.api.java.JavaRDDLike[T, R]](
-    dstream: JavaDStreamLike[T, This, R]) = {
+  def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]](
+    dstream: JavaDStreamLike[T, This, R]) =
+  {
     implicit val cm: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
     val ostream = new TestOutputStream(dstream.dstream,
diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
similarity index 98%
rename from streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
rename to streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index 67e3e0cd30c57a7544bce9a57c9bef44c0efbd2a..11586f72b6ddb081be0e4bc9e29f989ada9e7876 100644
--- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
-import spark.streaming.StreamingContext._
+import org.apache.spark.streaming.StreamingContext._
 import scala.runtime.RichInt
 import util.ManualClock
 
@@ -26,7 +26,7 @@ class BasicOperationsSuite extends TestSuiteBase {
   override def framework() = "BasicOperationsSuite"
 
   before {
-    System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+    System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
   }
 
   after {
diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
similarity index 98%
rename from streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
rename to streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
index 8c639648f00971278cdd650f9559a5d8547e8e1e..a327de80b3eb3a0f98bcbd2486e0580225b93a8a 100644
--- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 import dstream.FileInputDStream
-import spark.streaming.StreamingContext._
+import org.apache.spark.streaming.StreamingContext._
 import java.io.File
 import runtime.RichInt
 import org.scalatest.BeforeAndAfter
@@ -36,7 +36,7 @@ import com.google.common.io.Files
  */
 class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
 
-  System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+  System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
 
   before {
     FileUtils.deleteDirectory(new File(checkpointDir))
@@ -63,7 +63,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
 
     assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second")
 
-    System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+    System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
 
     val stateStreamCheckpointInterval = Seconds(1)
 
diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala
similarity index 93%
rename from streaming/src/test/scala/spark/streaming/FailureSuite.scala
rename to streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala
index 7fc649fe2705f409100e70c711ca05d2088c62af..6337c5359c3dcac1d8206d8881b6b28864484522 100644
--- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
-import spark.Logging
-import spark.streaming.util.MasterFailureTest
+import org.apache.spark.Logging
+import org.apache.spark.streaming.util.MasterFailureTest
 import StreamingContext._
 
 import org.scalatest.{FunSuite, BeforeAndAfter}
diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
similarity index 97%
rename from streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
rename to streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index 1c5419b16dab12780d3924751fa989cda19588b5..42e3e51e3fa15b18de981fe46f40fe999786a276 100644
--- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
 import akka.actor.Actor
 import akka.actor.IO
@@ -29,9 +29,9 @@ import java.io.{File, BufferedWriter, OutputStreamWriter}
 import java.util.concurrent.{TimeUnit, ArrayBlockingQueue}
 import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
 import util.ManualClock
-import spark.storage.StorageLevel
-import spark.streaming.receivers.Receiver
-import spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.receivers.Receiver
+import org.apache.spark.Logging
 import scala.util.Random
 import org.apache.commons.io.FileUtils
 import org.scalatest.BeforeAndAfter
@@ -52,7 +52,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
   override def checkpointDir = "checkpoint"
 
   before {
-    System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+    System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
   }
 
   after {
@@ -207,7 +207,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     FileUtils.deleteDirectory(testDir)
 
     // Enable manual clock back again for other tests
-    System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+    System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
   }
 
 
diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
similarity index 98%
rename from streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
rename to streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index cb34b5a7cc48a3fded3ba91b0489f448b3a796ce..31c2fa0208e1529271c51f2d6bed9ebe61df31fd 100644
--- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
-import spark.streaming.dstream.{InputDStream, ForEachDStream}
-import spark.streaming.util.ManualClock
+import org.apache.spark.streaming.dstream.{InputDStream, ForEachDStream}
+import org.apache.spark.streaming.util.ManualClock
 
-import spark.{RDD, Logging}
+import org.apache.spark.{RDD, Logging}
 
 import collection.mutable.ArrayBuffer
 import collection.mutable.SynchronizedBuffer
diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
similarity index 98%
rename from streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
rename to streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
index 894b765fc625f68b458ab4ccd6649829f3720b09..f50e05c0d883b97efbd0a92e7a3b9aedb9526fbb 100644
--- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package spark.streaming
+package org.apache.spark.streaming
 
-import spark.streaming.StreamingContext._
+import org.apache.spark.streaming.StreamingContext._
 import collection.mutable.ArrayBuffer
 
 class WindowOperationsSuite extends TestSuiteBase {
 
-  System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+  System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
 
   override def framework = "WindowOperationsSuite"
 
diff --git a/tools/pom.xml b/tools/pom.xml
index 95b5e80e5b1bc8c9839afc15747aa7940420cb11..664bf7dcbdb3e4b71259665bebe723067c582434 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -18,13 +18,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-tools</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project Tools</name>
@@ -32,12 +32,12 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-core</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>org.spark-project</groupId>
+      <groupId>org.apache.spark</groupId>
       <artifactId>spark-streaming</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
similarity index 64%
rename from tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala
rename to tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
index f45d0b281cf0a1a38def95993c6c6569f00abc42..50335e5736ba6ddd9219d8b17a66e70d0b24369d 100644
--- a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package spark.tools
+package org.apache.spark.tools
 
 import java.lang.reflect.Method
 
 import scala.collection.mutable.ArrayBuffer
 
-import spark._
-import spark.api.java._
-import spark.rdd.OrderedRDDFunctions
-import spark.streaming.{PairDStreamFunctions, DStream, StreamingContext}
-import spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext}
+import org.apache.spark._
+import org.apache.spark.api.java._
+import org.apache.spark.rdd.OrderedRDDFunctions
+import org.apache.spark.streaming.{PairDStreamFunctions, DStream, StreamingContext}
+import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext}
 
 
 private[spark] abstract class SparkType(val name: String)
@@ -129,7 +129,7 @@ object JavaAPICompletenessChecker {
       // TODO: the JavaStreamingContext API accepts Array arguments
       // instead of Lists, so this isn't a trivial translation / sub:
       "scala.collection.Seq" -> "java.util.List",
-      "scala.Function2" -> "spark.api.java.function.Function2",
+      "scala.Function2" -> "org.apache.spark.api.java.function.Function2",
       "scala.collection.Iterator" -> "java.util.Iterator",
       "scala.collection.mutable.Queue" -> "java.util.Queue",
       "double" -> "java.lang.Double"
@@ -139,7 +139,7 @@ object JavaAPICompletenessChecker {
       scalaType match {
         case ParameterizedType(name, parameters, typebounds) =>
           name match {
-            case "spark.RDD" =>
+            case "org.apache.spark.RDD" =>
               if (parameters(0).name == classOf[Tuple2[_, _]].getName) {
                 val tupleParams =
                   parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs)
@@ -147,13 +147,13 @@ object JavaAPICompletenessChecker {
               } else {
                 ParameterizedType(classOf[JavaRDD[_]].getName, parameters.map(applySubs))
               }
-            case "spark.streaming.DStream" =>
+            case "org.apache.spark.streaming.DStream" =>
               if (parameters(0).name == classOf[Tuple2[_, _]].getName) {
                 val tupleParams =
                   parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs)
-                ParameterizedType("spark.streaming.api.java.JavaPairDStream", tupleParams)
+                ParameterizedType("org.apache.spark.streaming.api.java.JavaPairDStream", tupleParams)
               } else {
-                ParameterizedType("spark.streaming.api.java.JavaDStream",
+                ParameterizedType("org.apache.spark.streaming.api.java.JavaDStream",
                   parameters.map(applySubs))
               }
             case "scala.Option" => {
@@ -167,14 +167,14 @@ object JavaAPICompletenessChecker {
               val firstParamName = parameters.last.name
               if (firstParamName.startsWith("scala.collection.Traversable") ||
                 firstParamName.startsWith("scala.collection.Iterator")) {
-                ParameterizedType("spark.api.java.function.FlatMapFunction",
+                ParameterizedType("org.apache.spark.api.java.function.FlatMapFunction",
                   Seq(parameters(0),
                     parameters.last.asInstanceOf[ParameterizedType].parameters(0)).map(applySubs))
               } else if (firstParamName == "scala.runtime.BoxedUnit") {
-                ParameterizedType("spark.api.java.function.VoidFunction",
+                ParameterizedType("org.apache.spark.api.java.function.VoidFunction",
                   parameters.dropRight(1).map(applySubs))
               } else {
-                ParameterizedType("spark.api.java.function.Function", parameters.map(applySubs))
+                ParameterizedType("org.apache.spark.api.java.function.Function", parameters.map(applySubs))
               }
             case _ =>
               ParameterizedType(renameSubstitutions.getOrElse(name, name),
@@ -211,85 +211,85 @@ object JavaAPICompletenessChecker {
     // This list also includes a few methods that are only used by the web UI or other
     // internal Spark components.
     val excludedNames = Seq(
-      "spark.RDD.origin",
-      "spark.RDD.elementClassManifest",
-      "spark.RDD.checkpointData",
-      "spark.RDD.partitioner",
-      "spark.RDD.partitions",
-      "spark.RDD.firstParent",
-      "spark.RDD.doCheckpoint",
-      "spark.RDD.markCheckpointed",
-      "spark.RDD.clearDependencies",
-      "spark.RDD.getDependencies",
-      "spark.RDD.getPartitions",
-      "spark.RDD.dependencies",
-      "spark.RDD.getPreferredLocations",
-      "spark.RDD.collectPartitions",
-      "spark.RDD.computeOrReadCheckpoint",
-      "spark.PairRDDFunctions.getKeyClass",
-      "spark.PairRDDFunctions.getValueClass",
-      "spark.SparkContext.stringToText",
-      "spark.SparkContext.makeRDD",
-      "spark.SparkContext.runJob",
-      "spark.SparkContext.runApproximateJob",
-      "spark.SparkContext.clean",
-      "spark.SparkContext.metadataCleaner",
-      "spark.SparkContext.ui",
-      "spark.SparkContext.newShuffleId",
-      "spark.SparkContext.newRddId",
-      "spark.SparkContext.cleanup",
-      "spark.SparkContext.receiverJobThread",
-      "spark.SparkContext.getRDDStorageInfo",
-      "spark.SparkContext.addedFiles",
-      "spark.SparkContext.addedJars",
-      "spark.SparkContext.persistentRdds",
-      "spark.SparkContext.executorEnvs",
-      "spark.SparkContext.checkpointDir",
-      "spark.SparkContext.getSparkHome",
-      "spark.SparkContext.executorMemoryRequested",
-      "spark.SparkContext.getExecutorStorageStatus",
-      "spark.streaming.DStream.generatedRDDs",
-      "spark.streaming.DStream.zeroTime",
-      "spark.streaming.DStream.rememberDuration",
-      "spark.streaming.DStream.storageLevel",
-      "spark.streaming.DStream.mustCheckpoint",
-      "spark.streaming.DStream.checkpointDuration",
-      "spark.streaming.DStream.checkpointData",
-      "spark.streaming.DStream.graph",
-      "spark.streaming.DStream.isInitialized",
-      "spark.streaming.DStream.parentRememberDuration",
-      "spark.streaming.DStream.initialize",
-      "spark.streaming.DStream.validate",
-      "spark.streaming.DStream.setContext",
-      "spark.streaming.DStream.setGraph",
-      "spark.streaming.DStream.remember",
-      "spark.streaming.DStream.getOrCompute",
-      "spark.streaming.DStream.generateJob",
-      "spark.streaming.DStream.clearOldMetadata",
-      "spark.streaming.DStream.addMetadata",
-      "spark.streaming.DStream.updateCheckpointData",
-      "spark.streaming.DStream.restoreCheckpointData",
-      "spark.streaming.DStream.isTimeValid",
-      "spark.streaming.StreamingContext.nextNetworkInputStreamId",
-      "spark.streaming.StreamingContext.networkInputTracker",
-      "spark.streaming.StreamingContext.checkpointDir",
-      "spark.streaming.StreamingContext.checkpointDuration",
-      "spark.streaming.StreamingContext.receiverJobThread",
-      "spark.streaming.StreamingContext.scheduler",
-      "spark.streaming.StreamingContext.initialCheckpoint",
-      "spark.streaming.StreamingContext.getNewNetworkStreamId",
-      "spark.streaming.StreamingContext.validate",
-      "spark.streaming.StreamingContext.createNewSparkContext",
-      "spark.streaming.StreamingContext.rddToFileName",
-      "spark.streaming.StreamingContext.getSparkCheckpointDir",
-      "spark.streaming.StreamingContext.env",
-      "spark.streaming.StreamingContext.graph",
-      "spark.streaming.StreamingContext.isCheckpointPresent"
+      "org.apache.spark.RDD.origin",
+      "org.apache.spark.RDD.elementClassManifest",
+      "org.apache.spark.RDD.checkpointData",
+      "org.apache.spark.RDD.partitioner",
+      "org.apache.spark.RDD.partitions",
+      "org.apache.spark.RDD.firstParent",
+      "org.apache.spark.RDD.doCheckpoint",
+      "org.apache.spark.RDD.markCheckpointed",
+      "org.apache.spark.RDD.clearDependencies",
+      "org.apache.spark.RDD.getDependencies",
+      "org.apache.spark.RDD.getPartitions",
+      "org.apache.spark.RDD.dependencies",
+      "org.apache.spark.RDD.getPreferredLocations",
+      "org.apache.spark.RDD.collectPartitions",
+      "org.apache.spark.RDD.computeOrReadCheckpoint",
+      "org.apache.spark.PairRDDFunctions.getKeyClass",
+      "org.apache.spark.PairRDDFunctions.getValueClass",
+      "org.apache.spark.SparkContext.stringToText",
+      "org.apache.spark.SparkContext.makeRDD",
+      "org.apache.spark.SparkContext.runJob",
+      "org.apache.spark.SparkContext.runApproximateJob",
+      "org.apache.spark.SparkContext.clean",
+      "org.apache.spark.SparkContext.metadataCleaner",
+      "org.apache.spark.SparkContext.ui",
+      "org.apache.spark.SparkContext.newShuffleId",
+      "org.apache.spark.SparkContext.newRddId",
+      "org.apache.spark.SparkContext.cleanup",
+      "org.apache.spark.SparkContext.receiverJobThread",
+      "org.apache.spark.SparkContext.getRDDStorageInfo",
+      "org.apache.spark.SparkContext.addedFiles",
+      "org.apache.spark.SparkContext.addedJars",
+      "org.apache.spark.SparkContext.persistentRdds",
+      "org.apache.spark.SparkContext.executorEnvs",
+      "org.apache.spark.SparkContext.checkpointDir",
+      "org.apache.spark.SparkContext.getSparkHome",
+      "org.apache.spark.SparkContext.executorMemoryRequested",
+      "org.apache.spark.SparkContext.getExecutorStorageStatus",
+      "org.apache.spark.streaming.DStream.generatedRDDs",
+      "org.apache.spark.streaming.DStream.zeroTime",
+      "org.apache.spark.streaming.DStream.rememberDuration",
+      "org.apache.spark.streaming.DStream.storageLevel",
+      "org.apache.spark.streaming.DStream.mustCheckpoint",
+      "org.apache.spark.streaming.DStream.checkpointDuration",
+      "org.apache.spark.streaming.DStream.checkpointData",
+      "org.apache.spark.streaming.DStream.graph",
+      "org.apache.spark.streaming.DStream.isInitialized",
+      "org.apache.spark.streaming.DStream.parentRememberDuration",
+      "org.apache.spark.streaming.DStream.initialize",
+      "org.apache.spark.streaming.DStream.validate",
+      "org.apache.spark.streaming.DStream.setContext",
+      "org.apache.spark.streaming.DStream.setGraph",
+      "org.apache.spark.streaming.DStream.remember",
+      "org.apache.spark.streaming.DStream.getOrCompute",
+      "org.apache.spark.streaming.DStream.generateJob",
+      "org.apache.spark.streaming.DStream.clearOldMetadata",
+      "org.apache.spark.streaming.DStream.addMetadata",
+      "org.apache.spark.streaming.DStream.updateCheckpointData",
+      "org.apache.spark.streaming.DStream.restoreCheckpointData",
+      "org.apache.spark.streaming.DStream.isTimeValid",
+      "org.apache.spark.streaming.StreamingContext.nextNetworkInputStreamId",
+      "org.apache.spark.streaming.StreamingContext.networkInputTracker",
+      "org.apache.spark.streaming.StreamingContext.checkpointDir",
+      "org.apache.spark.streaming.StreamingContext.checkpointDuration",
+      "org.apache.spark.streaming.StreamingContext.receiverJobThread",
+      "org.apache.spark.streaming.StreamingContext.scheduler",
+      "org.apache.spark.streaming.StreamingContext.initialCheckpoint",
+      "org.apache.spark.streaming.StreamingContext.getNewNetworkStreamId",
+      "org.apache.spark.streaming.StreamingContext.validate",
+      "org.apache.spark.streaming.StreamingContext.createNewSparkContext",
+      "org.apache.spark.streaming.StreamingContext.rddToFileName",
+      "org.apache.spark.streaming.StreamingContext.getSparkCheckpointDir",
+      "org.apache.spark.streaming.StreamingContext.env",
+      "org.apache.spark.streaming.StreamingContext.graph",
+      "org.apache.spark.streaming.StreamingContext.isCheckpointPresent"
     )
     val excludedPatterns = Seq(
-      """^spark\.SparkContext\..*To.*Functions""",
-      """^spark\.SparkContext\..*WritableConverter""",
-      """^spark\.SparkContext\..*To.*Writable"""
+      """^org\.apache\.spark\.SparkContext\..*To.*Functions""",
+      """^org\.apache\.spark\.SparkContext\..*WritableConverter""",
+      """^org\.apache\.spark\.SparkContext\..*To.*Writable"""
     ).map(_.r)
     lazy val excludedByPattern =
       !excludedPatterns.map(_.findFirstIn(name)).filter(_.isDefined).isEmpty
@@ -298,7 +298,7 @@ object JavaAPICompletenessChecker {
 
   private def isExcludedByInterface(method: Method): Boolean = {
     val excludedInterfaces =
-      Set("spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil")
+      Set("org.apache.spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil")
     def toComparisionKey(method: Method) =
       (method.getReturnType, method.getName, method.getGenericReturnType)
     val interfaces = method.getDeclaringClass.getInterfaces.filter { i =>
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 07dd170eae54fcbe58d97cbe50acc78e6814a244..a2afbeabff443a9a1a0f4608d0ea0717f7959b6c 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -18,13 +18,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <parent>
-    <groupId>org.spark-project</groupId>
+    <groupId>org.apache.spark</groupId>
     <artifactId>spark-parent</artifactId>
     <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
-  <groupId>org.spark-project</groupId>
+  <groupId>org.apache.spark</groupId>
   <artifactId>spark-yarn</artifactId>
   <packaging>jar</packaging>
   <name>Spark Project YARN Support</name>
@@ -81,7 +81,7 @@
       <id>hadoop2-yarn</id>
       <dependencies>
         <dependency>
-          <groupId>org.spark-project</groupId>
+          <groupId>org.apache.spark</groupId>
           <artifactId>spark-core</artifactId>
           <version>${project.version}</version>
         </dependency>
diff --git a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
similarity index 99%
rename from yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala
rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 0f3b6bc1a65d0c77e6d84d330529705ff5fd1238..139a977a0311f13384b948bac273c48b5a267254 100644
--- a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.yarn
+package org.apache.spark.deploy.yarn
 
 import java.net.Socket
 import java.util.concurrent.CopyOnWriteArrayList
@@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
 import scala.collection.JavaConversions._
-import spark.{SparkContext, Logging, Utils}
+import org.apache.spark.{SparkContext, Logging, Utils}
 import org.apache.hadoop.security.UserGroupInformation
 import java.security.PrivilegedExceptionAction
 
diff --git a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
similarity index 97%
rename from yarn/src/main/scala/spark/deploy/yarn/ApplicationMasterArguments.scala
rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
index 8de44b1f6617dabc7994acd26ae66c47b5fb3b4d..f47e23b63f4eaaae9f2b54213a1e68ef3af8d6a2 100644
--- a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMasterArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.deploy.yarn
+package org.apache.spark.deploy.yarn
 
-import spark.util.IntParam
+import org.apache.spark.util.IntParam
 import collection.mutable.ArrayBuffer
 
 class ApplicationMasterArguments(val args: Array[String]) {
diff --git a/yarn/src/main/scala/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
similarity index 99%
rename from yarn/src/main/scala/spark/deploy/yarn/Client.scala
rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index eb2a8cc6420765c88d9c87957f55f8540080f59f..48e737ed79774a38adc4dd408e38947c5283c74d 100644
--- a/yarn/src/main/scala/spark/deploy/yarn/Client.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.yarn
+package org.apache.spark.deploy.yarn
 
 import java.net.{InetSocketAddress, URI}
 import java.nio.ByteBuffer
@@ -33,10 +33,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
 import org.apache.hadoop.yarn.ipc.YarnRPC
 import scala.collection.mutable.HashMap
 import scala.collection.JavaConversions._
-import spark.{Logging, Utils}
+import org.apache.spark.{Logging, Utils}
 import org.apache.hadoop.yarn.util.{Apps, Records, ConverterUtils}
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.SparkHadoopUtil
 
 class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
   
diff --git a/yarn/src/main/scala/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
similarity index 95%
rename from yarn/src/main/scala/spark/deploy/yarn/ClientArguments.scala
rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index 67aff03781cb40527edcf3f962c34b698896edf9..6cbfadc23be93d609fb349c8ba135038c9f7ac70 100644
--- a/yarn/src/main/scala/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package spark.deploy.yarn
+package org.apache.spark.deploy.yarn
 
-import spark.util.MemoryParam
-import spark.util.IntParam
+import org.apache.spark.util.MemoryParam
+import org.apache.spark.util.IntParam
 import collection.mutable.{ArrayBuffer, HashMap}
-import spark.scheduler.{InputFormatInfo, SplitInfo}
+import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
 
 // TODO: Add code and support for ensuring that yarn resource 'asks' are location aware !
 class ClientArguments(val args: Array[String]) {
diff --git a/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
similarity index 98%
rename from yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala
rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index 0e1fd9b680e3987a5f46280af4a7697de25648b1..72dcf7178eb304eedea6fe9738fa901d0c13eb78 100644
--- a/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package spark.deploy.yarn
+package org.apache.spark.deploy.yarn
 
 import java.net.URI
 import java.nio.ByteBuffer
@@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
 import scala.collection.JavaConversions._
 import scala.collection.mutable.HashMap
 
-import spark.{Logging, Utils}
+import org.apache.spark.{Logging, Utils}
 
 class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String,
     slaveId: String, hostname: String, workerMemory: Int, workerCores: Int) 
@@ -119,7 +119,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
       // TODO: If the OOM is not recoverable by rescheduling it on different node, then do 'something' to fail job ... akin to blacklisting trackers in mapred ?
       " -XX:OnOutOfMemoryError='kill %p' " +
       JAVA_OPTS +
-      " spark.executor.StandaloneExecutorBackend " +
+      " org.apache.spark.executor.StandaloneExecutorBackend " +
       masterAddress + " " +
       slaveId + " " +
       hostname + " " +
diff --git a/yarn/src/main/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
similarity index 99%
rename from yarn/src/main/scala/spark/deploy/yarn/YarnAllocationHandler.scala
rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index b0af8baf08c152293e96cd87aef2cda27e39bba9..26ff214e122420314938f82dd634d719a7d99817 100644
--- a/yarn/src/main/scala/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-package spark.deploy.yarn
+package org.apache.spark.deploy.yarn
 
-import spark.{Logging, Utils}
-import spark.scheduler.SplitInfo
+import org.apache.spark.{Logging, Utils}
+import org.apache.spark.scheduler.SplitInfo
 import scala.collection
 import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId, ContainerId, Priority, Resource, ResourceRequest, ContainerStatus, Container}
-import spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend}
+import org.apache.spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend}
 import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse}
 import org.apache.hadoop.yarn.util.{RackResolver, Records}
 import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
diff --git a/yarn/src/main/scala/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
similarity index 95%
rename from yarn/src/main/scala/spark/deploy/yarn/YarnSparkHadoopUtil.scala
rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
index 77c4ee7f3f67f9afec64bc3aa978c09ce43e35be..ca2f1e2565b9a8a7956ffe00e2b453779956853a 100644
--- a/yarn/src/main/scala/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
@@ -15,9 +15,9 @@
  * limitations under the License.
  */
 
-package spark.deploy.yarn
+package org.apache.spark.deploy.yarn
 
-import spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.SparkHadoopUtil
 import collection.mutable.HashMap
 import org.apache.hadoop.mapred.JobConf
 import org.apache.hadoop.security.UserGroupInformation
diff --git a/yarn/src/main/scala/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
similarity index 93%
rename from yarn/src/main/scala/spark/scheduler/cluster/YarnClusterScheduler.scala
rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
index bb58353e0cc68e657b20c5e06478d12545fb08ee..3828ddfc4f14afe0d2bd0665b3476c79ad0cb648 100644
--- a/yarn/src/main/scala/spark/scheduler/cluster/YarnClusterScheduler.scala
+++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-package spark.scheduler.cluster
+package org.apache.spark.scheduler.cluster
 
-import spark._
-import spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler}
+import org.apache.spark._
+import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler}
 import org.apache.hadoop.conf.Configuration
 
 /**