You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2013/07/17 02:30:58 UTC

[01/50] [abbrv] git commit: Removed unit test of nonexistent function Utils.lastNBytes

Updated Branches:
  refs/heads/master d733527bb -> 87d586e4d


Removed unit test of nonexistent function Utils.lastNBytes


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/73984b96
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/73984b96
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/73984b96

Branch: refs/heads/master
Commit: 73984b96a8450674b472676eaa855cc2df68a754
Parents: 5c67ca0
Author: Karen Feng <ka...@gmail.com>
Authored: Fri Jul 12 14:26:56 2013 -0700
Committer: Karen Feng <ka...@gmail.com>
Committed: Fri Jul 12 14:26:56 2013 -0700

----------------------------------------------------------------------
 core/src/test/scala/spark/UtilsSuite.scala | 27 -------------------------
 1 file changed, 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/73984b96/core/src/test/scala/spark/UtilsSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala
index d83a030..1e1260f 100644
--- a/core/src/test/scala/spark/UtilsSuite.scala
+++ b/core/src/test/scala/spark/UtilsSuite.scala
@@ -91,33 +91,6 @@ class UtilsSuite extends FunSuite {
     assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h")
   }
 
-  test("reading last n bytes of a file") {
-    val tmpDir = Files.createTempDir()
-
-    // File smaller than limit
-    val f1Path = tmpDir + "/f1"
-    val f1 = new FileOutputStream(f1Path)
-    f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8))
-    f1.close()
-    assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd")
-
-    // File larger than limit
-    val f2Path = tmpDir + "/f2"
-    val f2 = new FileOutputStream(f2Path)
-    f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8))
-    f2.close()
-    assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n")
-
-    // Request limit too
-    val f3Path = tmpDir + "/f2"
-    val f3 = new FileOutputStream(f3Path)
-    f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8))
-    f3.close()
-    assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n")
-
-    FileUtils.deleteDirectory(tmpDir)
-  }
-
   test("reading offset bytes of a file") {
     val tmpDir2 = Files.createTempDir()
     val f1Path = tmpDir2 + "/f1"


[33/50] [abbrv] git commit: Removed job UI column, linked description to job UI

Posted by ma...@apache.org.
Removed job UI column, linked description to job UI


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/6dc7c9bf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/6dc7c9bf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/6dc7c9bf

Branch: refs/heads/master
Commit: 6dc7c9bfb17cefdf162e86c8b52a4ffc8b59efaf
Parents: fbf5aa7
Author: Karen Feng <ka...@gmail.com>
Authored: Mon Jul 15 16:33:50 2013 -0700
Committer: Karen Feng <ka...@gmail.com>
Committed: Mon Jul 15 16:33:50 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6dc7c9bf/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
index 434f600..5e3c5e0 100644
--- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
@@ -26,8 +26,8 @@ private[spark] class IndexPage(parent: MasterWebUI) {
     val workers = state.workers.sortBy(_.id)
     val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers)
 
-    val appHeaders = Seq("ID", "Job UI", "Description", "Cores", "Memory per Node", "Submit Time",
-      "User", "State", "Duration")
+    val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User",
+      "State", "Duration")
     val activeApps = state.activeApps.sortBy(_.startTime).reverse
     val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps)
     val completedApps = state.completedApps.sortBy(_.endTime).reverse
@@ -104,9 +104,8 @@ private[spark] class IndexPage(parent: MasterWebUI) {
         <a href={"app?appId=" + app.id}>{app.id}</a>
       </td>
       <td>
-        <a href={app.appUiUrl}>{app.appUiUrl}</a>
+        <a href={app.appUiUrl}>{app.desc.name}</a>
       </td>
-      <td>{app.desc.name}</td>
       <td>
         {app.coresGranted}
       </td>


[25/50] [abbrv] git commit: Add an option to disable reference tracking in Kryo

Posted by ma...@apache.org.
Add an option to disable reference tracking in Kryo


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/d47c16f7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/d47c16f7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/d47c16f7

Branch: refs/heads/master
Commit: d47c16f78d5cb935bd4022c9bed8376691371682
Parents: 238d0e6
Author: Matei Zaharia <ma...@gmail.com>
Authored: Mon Jul 15 01:55:54 2013 +0000
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Mon Jul 15 01:55:54 2013 +0000

----------------------------------------------------------------------
 core/src/main/scala/spark/KryoSerializer.scala |  4 ++++
 docs/configuration.md                          | 12 +++++++++++-
 2 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d47c16f7/core/src/main/scala/spark/KryoSerializer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala
index d723ab7..c7dbcc6 100644
--- a/core/src/main/scala/spark/KryoSerializer.scala
+++ b/core/src/main/scala/spark/KryoSerializer.scala
@@ -210,6 +210,10 @@ class KryoSerializer extends spark.serializer.Serializer with Logging {
       val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]
       reg.registerClasses(kryo)
     }
+
+    // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops
+    kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean)
+
     kryo
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/d47c16f7/docs/configuration.md
----------------------------------------------------------------------
diff --git a/docs/configuration.md b/docs/configuration.md
index 5a80510..5c06897 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -198,8 +198,18 @@ Apart from these, the following properties are also available, and may be useful
   </td>
 </tr>
 <tr>
+  <td>spark.kryo.referenceTracking</td>
+  <td>true</td>
+  <td>
+    Whether to track references to the same object when serializing data with Kryo, which is
+    necessary if your object graphs have loops and useful for efficiency if they contain multiple
+    copies of the same object. Can be disabled to improve performance if you know this is not the
+    case.
+  </td>
+</tr>
+<tr>
   <td>spark.kryoserializer.buffer.mb</td>
-  <td>32</td>
+  <td>2</td>
   <td>
     Maximum object size to allow within Kryo (the library needs to create a buffer at least as
     large as the largest single object you'll serialize). Increase this if you get a "buffer limit


[13/50] [abbrv] git commit: Merge branch 'master' of github.com:mesos/spark

Posted by ma...@apache.org.
Merge branch 'master' of github.com:mesos/spark


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/668b0dc6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/668b0dc6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/668b0dc6

Branch: refs/heads/master
Commit: 668b0dc6a7fac85cde93d96adf966bbf98275018
Parents: cd28d9c 89e8549
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Sat Jul 13 19:10:46 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Sat Jul 13 19:10:46 2013 -0700

----------------------------------------------------------------------
 project/SparkBuild.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/668b0dc6/project/SparkBuild.scala
----------------------------------------------------------------------


[22/50] [abbrv] git commit: Merge pull request #689 from BlackNiuza/application_status

Posted by ma...@apache.org.
Merge pull request #689 from BlackNiuza/application_status

Bug fix: SPARK-796

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/c7877d5e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/c7877d5e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/c7877d5e

Branch: refs/heads/master
Commit: c7877d5e16e5441a9891d5eda67a8a062278b103
Parents: 10c0593 00556a9
Author: Matei Zaharia <ma...@gmail.com>
Authored: Sun Jul 14 12:58:13 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Sun Jul 14 12:58:13 2013 -0700

----------------------------------------------------------------------
 .../spark/deploy/yarn/ApplicationMaster.scala   | 58 ++++++++++++++------
 1 file changed, 40 insertions(+), 18 deletions(-)
----------------------------------------------------------------------



[15/50] [abbrv] git commit: Determine Spark core classes better in getCallSite

Posted by ma...@apache.org.
Determine Spark core classes better in getCallSite


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/a44a7b12
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/a44a7b12
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/a44a7b12

Branch: refs/heads/master
Commit: a44a7b123862202cf97f2de7a96aeaf29a93002a
Parents: e271fde
Author: Matei Zaharia <ma...@gmail.com>
Authored: Sun Jul 14 07:23:09 2013 +0000
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Sun Jul 14 07:23:09 2013 +0000

----------------------------------------------------------------------
 core/src/main/scala/spark/Utils.scala | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/a44a7b12/core/src/main/scala/spark/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala
index c6a3f97..a36186b 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/spark/Utils.scala
@@ -579,8 +579,15 @@ private object Utils extends Logging {
     output.toString
   }
 
+  /** 
+   * A regular expression to match classes of the "core" Spark API that we want to skip when
+   * finding the call site of a method.
+   */
+  private val SPARK_CLASS_REGEX = """^spark(\.api\.java)?(\.rdd)?\.[A-Z]""".r
+
   private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String,
                                     val firstUserLine: Int, val firstUserClass: String)
+
   /**
    * When called inside a class in the spark package, returns the name of the user code class
    * (outside the spark package) that called into Spark, as well as which Spark method they called.
@@ -602,7 +609,7 @@ private object Utils extends Logging {
 
     for (el <- trace) {
       if (!finished) {
-        if (el.getClassName.startsWith("spark.") && !el.getClassName.startsWith("spark.examples.")) {
+        if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName) != None) {
           lastSparkMethod = if (el.getMethodName == "<init>") {
             // Spark method is a constructor; get its class name
             el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1)


[02/50] [abbrv] git commit: Show block locations in Web UI.

Posted by ma...@apache.org.
Show block locations in Web UI.

This fixes SPARK-769. Support is added for enumerating the locations of blocks
in the UI. There is also some minor cleanup in StorageUtils.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/6855338e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/6855338e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/6855338e

Branch: refs/heads/master
Commit: 6855338e1400638188358a7d7926eb86f668c160
Parents: 018d04c
Author: Patrick Wendell <pw...@gmail.com>
Authored: Fri Jul 12 19:24:16 2013 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Fri Jul 12 19:30:32 2013 -0700

----------------------------------------------------------------------
 .../main/scala/spark/storage/StorageUtils.scala    | 11 +++++++++--
 core/src/main/scala/spark/ui/storage/RDDPage.scala | 17 +++++++++++++----
 2 files changed, 22 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6855338e/core/src/main/scala/spark/storage/StorageUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala
index 950c0cd..3e7fa28 100644
--- a/core/src/main/scala/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/spark/storage/StorageUtils.scala
@@ -39,12 +39,19 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel,
 private[spark]
 object StorageUtils {
 
-  /* Given the current storage status of the BlockManager, returns information for each RDD */
-  def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus],
+  /* Returns RDD-level information, compiled from a list of StorageStatus objects */
+  def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus],
     sc: SparkContext) : Array[RDDInfo] = {
     rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc)
   }
 
+  /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */
+  def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = {
+     val blockLocationPairs = storageStatusList
+       .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort)))
+    blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap
+  }
+
   /* Given a list of BlockStatus objets, returns information for each RDD */
   def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus],
     sc: SparkContext) : Array[RDDInfo] = {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/6855338e/core/src/main/scala/spark/ui/storage/RDDPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala
index 0cb1e47..428db6f 100644
--- a/core/src/main/scala/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala
@@ -26,8 +26,14 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
     val workers = filteredStorageStatusList.map((prefix, _))
     val workerTable = listingTable(workerHeaders, workerRow, workers)
 
-    val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk")
-    val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1)
+    val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk",
+      "Locations")
+
+    val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1)
+    val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList)
+    val blocks = blockStatuses.map {
+      case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN")))
+    }
     val blockTable = listingTable(blockHeaders, blockRow, blocks)
 
     val content =
@@ -74,8 +80,8 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
     headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name, Jobs)
   }
 
-  def blockRow(blk: (String, BlockStatus)): Seq[Node] = {
-    val (id, block) = blk
+  def blockRow(row: (String, BlockStatus, Seq[String])): Seq[Node] = {
+    val (id, block, locations) = row
     <tr>
       <td>{id}</td>
       <td>
@@ -87,6 +93,9 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
       <td sorttable_customkey={block.diskSize.toString}>
         {Utils.memoryBytesToString(block.diskSize)}
       </td>
+      <td>
+        {locations.map(l => <span>{l}<br/></span>)}
+      </td>
     </tr>
   }
 


[26/50] [abbrv] git commit: Shuffle ratings in a more efficient way at start of ALS

Posted by ma...@apache.org.
Shuffle ratings in a more efficient way at start of ALS


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/4698a0d6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/4698a0d6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/4698a0d6

Branch: refs/heads/master
Commit: 4698a0d6886905ef21cbd52e108d0dcab3df12df
Parents: d47c16f
Author: Matei Zaharia <ma...@gmail.com>
Authored: Mon Jul 15 02:54:11 2013 +0000
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Mon Jul 15 02:54:11 2013 +0000

----------------------------------------------------------------------
 .../scala/spark/mllib/recommendation/ALS.scala    | 18 ++++++++++++++----
 1 file changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4698a0d6/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
index 2abaf2f..4c18cbd 100644
--- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
@@ -6,8 +6,10 @@ import scala.util.Sorting
 
 import spark.{HashPartitioner, Partitioner, SparkContext, RDD}
 import spark.storage.StorageLevel
+import spark.KryoRegistrator
 import spark.SparkContext._
 
+import com.esotericsoftware.kryo.Kryo
 import org.jblas.{DoubleMatrix, SimpleBlas, Solve}
 
 
@@ -98,8 +100,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
 
     val partitioner = new HashPartitioner(numBlocks)
 
-    val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) }
-    val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) }
+    val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, Rating(u, p, r)) }
+    val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, Rating(p, u, r)) }
 
     val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock)
     val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock)
@@ -179,12 +181,12 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
    * the users (or (blockId, (p, u, r)) for the products). We create these simultaneously to avoid
    * having to shuffle the (blockId, (u, p, r)) RDD twice, or to cache it.
    */
-  private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, (Int, Int, Double))])
+  private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)])
     : (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) =
   {
     val grouped = ratings.partitionBy(new HashPartitioner(numBlocks))
     val links = grouped.mapPartitionsWithIndex((blockId, elements) => {
-      val ratings = elements.map{case (k, t) => Rating(t._1, t._2, t._3)}.toArray
+      val ratings = elements.map{_._2}.toArray
       val inLinkBlock = makeInLinkBlock(numBlocks, ratings)
       val outLinkBlock = makeOutLinkBlock(numBlocks, ratings)
       Iterator.single((blockId, (inLinkBlock, outLinkBlock)))
@@ -383,6 +385,12 @@ object ALS {
     train(ratings, rank, iterations, 0.01, -1)
   }
 
+  private class ALSRegistrator extends KryoRegistrator {
+    override def registerClasses(kryo: Kryo) {
+      kryo.register(classOf[Rating])
+    }
+  }
+
   def main(args: Array[String]) {
     if (args.length != 5 && args.length != 6) {
       println("Usage: ALS <master> <ratings_file> <rank> <iterations> <output_dir> [<blocks>]")
@@ -392,6 +400,8 @@ object ALS {
       (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.kryo.registrator", classOf[ALSRegistrator].getName)
+    System.setProperty("spark.kryo.referenceTracking", "false")
     System.setProperty("spark.locality.wait", "10000")
     val sc = new SparkContext(master, "ALS")
     val ratings = sc.textFile(ratingsFile).map { line =>


[19/50] [abbrv] git commit: add spaces before curly braces and after for if conditions

Posted by ma...@apache.org.
add spaces before curly braces  and after for if conditions


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/00556a94
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/00556a94
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/00556a94

Branch: refs/heads/master
Commit: 00556a94c98577d1536d0c2b4606a9933069a584
Parents: ce18b50
Author: BlackNiuza <sh...@taobao.com>
Authored: Sun Jul 14 17:04:53 2013 +0800
Committer: BlackNiuza <sh...@taobao.com>
Committed: Sun Jul 14 17:04:53 2013 +0800

----------------------------------------------------------------------
 .../scala/spark/deploy/yarn/ApplicationMaster.scala | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/00556a94/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
index 68bb36d..6a0617c 100644
--- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
@@ -130,7 +130,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     val t = new Thread {
       override def run() {
         var successed = false
-        try{
+        try {
           // Copy
           var mainArgs: Array[String] = new Array[String](args.userArgs.size())
           args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size())
@@ -140,9 +140,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
           // It need shutdown hook to set SUCCEEDED
           successed = true
         } finally {
-          if(successed){
+          if (successed) {
             ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
-          }else{
+          } else {
             ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED)
           }
         }
@@ -190,7 +190,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
     logInfo("All workers have launched.")
 
     // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
-    if (userThread.isAlive){
+    if (userThread.isAlive) {
       // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
 
       val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
@@ -208,7 +208,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
 
     val t = new Thread {
       override def run() {
-        while (userThread.isAlive){
+        while (userThread.isAlive) {
           val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning
           if (missingWorkerCount > 0) {
             logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers")
@@ -250,7 +250,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
   def finishApplicationMaster(status: FinalApplicationStatus) {
 
     synchronized {
-      if(isFinished){
+      if (isFinished) {
         return
       }
       isFinished = true
@@ -276,7 +276,7 @@ object ApplicationMaster {
   private val ALLOCATOR_LOOP_WAIT_COUNT = 30
   def incrementAllocatorLoop(by: Int) {
     val count = yarnAllocatorLoop.getAndAdd(by)
-    if (count >= ALLOCATOR_LOOP_WAIT_COUNT){
+    if (count >= ALLOCATOR_LOOP_WAIT_COUNT) {
       yarnAllocatorLoop.synchronized {
         // to wake threads off wait ...
         yarnAllocatorLoop.notifyAll()
@@ -320,7 +320,7 @@ object ApplicationMaster {
 
     // Wait for initialization to complete and atleast 'some' nodes can get allocated
     yarnAllocatorLoop.synchronized {
-      while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT){
+      while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) {
         yarnAllocatorLoop.wait(1000L)
       }
     }


[49/50] [abbrv] git commit: Some missing license headers

Posted by ma...@apache.org.
Some missing license headers


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/4ff494de
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/4ff494de
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/4ff494de

Branch: refs/heads/master
Commit: 4ff494de20c36151dc29a60825d67e094d14acd4
Parents: af3c9d5
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Tue Jul 16 17:26:48 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Tue Jul 16 17:26:48 2013 -0700

----------------------------------------------------------------------
 bin/compute-classpath.cmd | 17 +++++++++++++++++
 bin/compute-classpath.sh  | 17 +++++++++++++++++
 2 files changed, 34 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4ff494de/bin/compute-classpath.cmd
----------------------------------------------------------------------
diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd
index 44826f3..835d1af 100644
--- a/bin/compute-classpath.cmd
+++ b/bin/compute-classpath.cmd
@@ -1,5 +1,22 @@
 @echo off
 
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
 rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
 rem script and the ExecutorRunner in standalone cluster mode.
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/4ff494de/bin/compute-classpath.sh
----------------------------------------------------------------------
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
index bd48b43..4c53964 100755
--- a/bin/compute-classpath.sh
+++ b/bin/compute-classpath.sh
@@ -1,5 +1,22 @@
 #!/bin/bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
 # script and the ExecutorRunner in standalone cluster mode.
 


[14/50] [abbrv] git commit: Fixed a delay scheduling bug in the YARN branch, found by Patrick

Posted by ma...@apache.org.
Fixed a delay scheduling bug in the YARN branch, found by Patrick


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/e271fde1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/e271fde1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/e271fde1

Branch: refs/heads/master
Commit: e271fde10b342216e33f4f45af73c5d103215cf2
Parents: 668b0dc
Author: root <ro...@ip-10-232-19-31.ec2.internal>
Authored: Sun Jul 14 06:24:29 2013 +0000
Committer: root <ro...@ip-10-232-19-31.ec2.internal>
Committed: Sun Jul 14 06:24:29 2013 +0000

----------------------------------------------------------------------
 .../spark/scheduler/cluster/ClusterTaskSetManager.scala   | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/e271fde1/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
index fe6420a..327d679 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
@@ -454,10 +454,10 @@ private[spark] class ClusterTaskSetManager(
           val taskId = sched.newTaskId()
           // Figure out whether this should count as a preferred launch
           val taskLocality =
-            if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else
-            if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else
-            if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else
-              TaskLocality.ANY
+            if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL
+            else if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL
+            else if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL
+            else  TaskLocality.ANY
           val prefStr = taskLocality.toString
           logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format(
             taskSet.id, index, taskId, execId, hostPort, prefStr))
@@ -467,7 +467,7 @@ private[spark] class ClusterTaskSetManager(
           val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality)
           taskInfos(taskId) = info
           taskAttempts(index) = info :: taskAttempts(index)
-          if (TaskLocality.NODE_LOCAL == taskLocality) {
+          if (taskLocality == TaskLocality.PROCESS_LOCAL || taskLocality == TaskLocality.NODE_LOCAL) {
             lastPreferredLaunchTime = time
           }
           // Serialize and return the task


[07/50] [abbrv] git commit: Merge pull request #697 from pwendell/block-locations

Posted by ma...@apache.org.
Merge pull request #697 from pwendell/block-locations

Show block locations in Web UI.

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/77c69ae5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/77c69ae5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/77c69ae5

Branch: refs/heads/master
Commit: 77c69ae5a0b94940008b72be6e60b688011daf95
Parents: 5a7835c 08150f1
Author: Matei Zaharia <ma...@gmail.com>
Authored: Fri Jul 12 23:05:21 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Fri Jul 12 23:05:21 2013 -0700

----------------------------------------------------------------------
 .../main/scala/spark/storage/StorageUtils.scala    | 11 +++++++++--
 core/src/main/scala/spark/ui/storage/RDDPage.scala | 17 +++++++++++++----
 2 files changed, 22 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[16/50] [abbrv] git commit: Cosmetic fixes to web UI

Posted by ma...@apache.org.
Cosmetic fixes to web UI


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/b91a218c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/b91a218c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/b91a218c

Branch: refs/heads/master
Commit: b91a218cea5a7ab4037675667922fc06bfec6fbf
Parents: a44a7b1
Author: Matei Zaharia <ma...@gmail.com>
Authored: Sun Jul 14 07:31:33 2013 +0000
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Sun Jul 14 07:31:33 2013 +0000

----------------------------------------------------------------------
 core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 4 ++--
 core/src/main/scala/spark/ui/jobs/IndexPage.scala          | 8 ++++----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b91a218c/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
index c6de2ba..7545ecf 100644
--- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
@@ -38,8 +38,8 @@ private[spark] class IndexPage(parent: MasterWebUI) {
         <div class="row">
           <div class="span12">
             <ul class="unstyled">
-              <li><strong>URL:</strong>{state.uri}</li>
-              <li><strong>Workers:</strong>{state.workers.size}</li>
+              <li><strong>URL:</strong> {state.uri}</li>
+              <li><strong>Workers:</strong> {state.workers.size}</li>
               <li><strong>Cores:</strong> {state.workers.map(_.cores).sum} Total,
                 {state.workers.map(_.coresUsed).sum} Used</li>
               <li><strong>Memory:</strong>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b91a218c/core/src/main/scala/spark/ui/jobs/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala
index 1e675ab..7907ab3 100644
--- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala
+++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala
@@ -29,10 +29,10 @@ private[spark] class IndexPage(parent: JobProgressUI) {
           <th>Stage Id</th>
           <th>Origin</th>
           <th>Submitted</th>
-          <td>Duration</td>
-          <td colspan="2">Tasks: Complete/Total</td>
-          <td>Shuffle Activity</td>
-          <td>Stored RDD</td>
+          <th>Duration</th>
+          <th colspan="2">Tasks: Complete/Total</th>
+          <th>Shuffle Activity</th>
+          <th>Stored RDD</th>
         </thead>
         <tbody>
           {rows.map(r => makeRow(r))}


[05/50] [abbrv] git commit: Merge pull request #696 from woggle/executor-env

Posted by ma...@apache.org.
Merge pull request #696 from woggle/executor-env

Pass executor env vars (e.g. SPARK_CLASSPATH) to compute-classpath.sh

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/71ccca0c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/71ccca0c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/71ccca0c

Branch: refs/heads/master
Commit: 71ccca0cc126c3dec12df10bca9e56a04a7a39e5
Parents: 90fc3f3 531a7e5
Author: Matei Zaharia <ma...@gmail.com>
Authored: Fri Jul 12 20:25:06 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Fri Jul 12 20:25:06 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/Utils.scala                   | 12 +++++++++---
 .../main/scala/spark/deploy/worker/ExecutorRunner.scala |  4 +++-
 2 files changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[41/50] [abbrv] Add Apache license headers and LICENSE and NOTICE files

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 2fb5bbd..7e6b06d 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
 <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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/Checkpoint.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
index 450e48d..1e4c1e3 100644
--- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.{Logging, Utils}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/DStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala
index 9be7926..684d3ab 100644
--- a/streaming/src/main/scala/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/spark/streaming/DStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.streaming.dstream._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala
index 6b0fade..399ca1c 100644
--- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala
+++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import org.apache.hadoop.fs.Path

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
index 3b33195..c09a332 100644
--- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import dstream.InputDStream

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/Duration.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala
index c213519..12a14e2 100644
--- a/streaming/src/main/scala/spark/streaming/Duration.scala
+++ b/streaming/src/main/scala/spark/streaming/Duration.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.Utils

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/Interval.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/spark/streaming/Interval.scala
index 6a8b817..b30cd96 100644
--- a/streaming/src/main/scala/spark/streaming/Interval.scala
+++ b/streaming/src/main/scala/spark/streaming/Interval.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 private[streaming]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/Job.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/Job.scala b/streaming/src/main/scala/spark/streaming/Job.scala
index 67bd838..ceb3f92 100644
--- a/streaming/src/main/scala/spark/streaming/Job.scala
+++ b/streaming/src/main/scala/spark/streaming/Job.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import java.util.concurrent.atomic.AtomicLong

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/JobManager.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala
index 7696c4a..a312306 100644
--- a/streaming/src/main/scala/spark/streaming/JobManager.scala
+++ b/streaming/src/main/scala/spark/streaming/JobManager.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.Logging 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
index b159d26..d4cf2e5 100644
--- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
index 8d0a83d..47bf07b 100644
--- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.streaming.StreamingContext._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/Scheduler.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala
index 1c4b22a..252cc2a 100644
--- a/streaming/src/main/scala/spark/streaming/Scheduler.scala
+++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import util.{ManualClock, RecurringTimer, Clock}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/StreamingContext.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
index 36b841a..ffd6562 100644
--- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import akka.actor.Props

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/Time.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/spark/streaming/Time.scala
index f14decf..ad5eab9 100644
--- a/streaming/src/main/scala/spark/streaming/Time.scala
+++ b/streaming/src/main/scala/spark/streaming/Time.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 /**
@@ -52,4 +69,4 @@ case class Time(private val millis: Long) {
 
 object Time {
   val ordering = Ordering.by((time: Time) => time.millis)
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
index 4d93f0a..7dcb1d7 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.api.java
 
 import spark.streaming.{Duration, Time, DStream}
@@ -82,4 +99,4 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM
 object JavaDStream {
   implicit def fromDStream[T: ClassManifest](dstream: DStream[T]): JavaDStream[T] =
     new JavaDStream[T](dstream)
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
index 548809a..3ab5c1f 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.api.java
 
 import java.util.{List => JList}
@@ -296,4 +313,4 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
   def checkpoint(interval: Duration) = {
     dstream.checkpoint(interval)
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
index 30240ca..ccd1556 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.api.java
 
 import java.util.{List => JList}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
index ed7b789..b7720ad 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.api.java
 
 import spark.streaming._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
index 4ef4bb7..99553d2 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.{RDD, Partitioner}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
index 41c3af4..0951370 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.RDD
@@ -16,4 +33,4 @@ class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T
   override def compute(validTime: Time): Option[RDD[T]] = {
     Some(rdd)
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
index 41b9bd9..de05361 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
index e993164..9d8c5c3 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Duration, DStream, Time}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
index cabd34f..78d7117 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Duration, DStream, Time}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
index a69af60..d13bebb 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Duration, DStream, Time}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
index c9644b3..4906f50 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.StreamingContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
index ee69ea5..7df537e 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
index b589cbd..06fda6f 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Duration, DStream, Time}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
index 3c5d43a..4dbdec4 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Time, Duration, StreamingContext, DStream}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
index 55d2957..6ee588a 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.Logging

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
index 848afec..af41a1b 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Duration, DStream, Time}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
index 6055aa6..8d8a616 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Duration, DStream, Time}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
index 20818a0..3fda84a 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Duration, DStream, Time}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
index 122a529..344b41c 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala
index 3c2a819..33f7cd0 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.StreamingContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
index da224ad..b269061 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
index 1b2fa56..236f74f 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.Logging

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
index 343b691..9626050 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.StreamingContext._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
index 1f9548b..83b57b2 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.{RDD, Partitioner}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
index 1408af0..5877b10 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.StreamingContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
index db62955..4b46613 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
index 99660d9..e7fbc5b 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
index ff7a58b..f09a8b9 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
index 00bad5d..3eaa9a7 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.streaming.{Duration, DStream, Time}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
index cbf0c88..fd24d61 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.dstream
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala
index 036c95a..2d9937e 100644
--- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala
+++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.receivers
 
 import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala
index 5533c3c..22d554e 100644
--- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala
+++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.receivers
 
 import akka.actor.Actor

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/util/Clock.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/util/Clock.scala b/streaming/src/main/scala/spark/streaming/util/Clock.scala
index 974651f..d9ac722 100644
--- a/streaming/src/main/scala/spark/streaming/util/Clock.scala
+++ b/streaming/src/main/scala/spark/streaming/util/Clock.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.util
 
 private[streaming]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
index 426a9b6..8ce5d8d 100644
--- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.util
 
 import spark.{Logging, RDD}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
index 03749d4..bf04120 100644
--- a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
+++ b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.util
 
 import spark.SparkContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
index bd0b0e7..5cc6ad9 100644
--- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
+++ b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.util
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
index 8e10276..7ecc442 100644
--- a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
+++ b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.util
 
 private[streaming]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/test/java/spark/streaming/JavaAPISuite.java
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java
index 4cf1058..3b93790 100644
--- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming;
 
 import com.google.common.base.Optional;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/test/java/spark/streaming/JavaTestUtils.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala
index 64a7e7c..f9d25db 100644
--- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala
+++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
@@ -64,4 +81,4 @@ object JavaTestUtils extends JavaTestBase {
 
 object JavaCheckpointTestUtils extends JavaTestBase {
   override def actuallyWait = true
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties
index 59c445e..063529a 100644
--- a/streaming/src/test/resources/log4j.properties
+++ b/streaming/src/test/resources/log4j.properties
@@ -1,4 +1,21 @@
-# Set everything to be logged to the file streaming/target/unit-tests.log 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
 log4j.rootCategory=INFO, file
 # log4j.appender.file=org.apache.log4j.FileAppender
 log4j.appender.file=org.apache.log4j.FileAppender

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
index 565089a..67e3e0c 100644
--- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.streaming.StreamingContext._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
index 607dea7..8c63964 100644
--- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import dstream.FileInputDStream

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/test/scala/spark/streaming/FailureSuite.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala
index a5fa7ab..7fc649f 100644
--- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.Logging

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
index b024fc9..1c5419b 100644
--- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import akka.actor.Actor

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
index ad6aa79..cb34b5a 100644
--- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.streaming.dstream.{InputDStream, ForEachDStream}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
----------------------------------------------------------------------
diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
index 80d8277..894b765 100644
--- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming
 
 import spark.streaming.StreamingContext._


[27/50] [abbrv] git commit: Adds app name in HTML page titles on job web UI: fixes SPARK-806

Posted by ma...@apache.org.
Adds app name in HTML page titles on job web UI: fixes SPARK-806


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/b2aaa119
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/b2aaa119
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/b2aaa119

Branch: refs/heads/master
Commit: b2aaa1199e7ecd8e1b2a9ddd8356b6393edafe6b
Parents: 4698a0d
Author: Karen Feng <ka...@gmail.com>
Authored: Mon Jul 15 11:44:42 2013 -0700
Committer: Karen Feng <ka...@gmail.com>
Committed: Mon Jul 15 11:44:42 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/ui/UIUtils.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b2aaa119/core/src/main/scala/spark/ui/UIUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala
index 36d9c47..fa46e24 100644
--- a/core/src/main/scala/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/spark/ui/UIUtils.scala
@@ -31,7 +31,7 @@ private[spark] object UIUtils {
         <link rel="stylesheet" href="/static/webui.css" type="text/css" />
         <link rel="stylesheet" href="/static/bootstrap-responsive.min.css" type="text/css" />
         <script src="/static/sorttable.js"></script>
-        <title>{title}</title>
+        <title>{sc.appName} - {title}</title>
         <style type="text/css">
           table.sortable thead {{ cursor: pointer; }}
         </style>


[50/50] [abbrv] git commit: Merge remote-tracking branch 'old/master'

Posted by ma...@apache.org.
Merge remote-tracking branch 'old/master'


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/87d586e4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/87d586e4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/87d586e4

Branch: refs/heads/master
Commit: 87d586e4da63e6e1875d9cac194c6f11e1cdc653
Parents: d733527 4ff494d
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Tue Jul 16 17:30:15 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Tue Jul 16 17:30:15 2013 -0700

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 LICENSE                                         |  229 +++-
 NOTICE                                          |    5 +
 README.md                                       |    2 +-
 bagel/pom.xml                                   |   17 +
 bagel/src/main/scala/spark/bagel/Bagel.scala    |   17 +
 .../spark/bagel/examples/PageRankUtils.scala    |   17 +
 .../bagel/examples/WikipediaPageRank.scala      |   17 +
 .../examples/WikipediaPageRankStandalone.scala  |   17 +
 bagel/src/test/resources/log4j.properties       |   19 +-
 bagel/src/test/scala/bagel/BagelSuite.scala     |   17 +
 bin/compute-classpath.cmd                       |   71 ++
 bin/compute-classpath.sh                        |  121 +++
 bin/slaves.sh                                   |    5 +-
 bin/spark-config.sh                             |   19 +-
 bin/spark-daemon.sh                             |    5 +-
 bin/spark-daemons.sh                            |   17 +
 bin/start-all.sh                                |   17 +
 bin/start-master.sh                             |   17 +
 bin/start-slave.sh                              |   20 +
 bin/start-slaves.sh                             |   17 +
 bin/stop-all.sh                                 |   17 +
 bin/stop-master.sh                              |   17 +
 bin/stop-slaves.sh                              |   17 +
 conf/fairscheduler.xml.template                 |   15 +
 conf/spark-env.sh.template                      |   18 +-
 core/pom.xml                                    |   45 +-
 .../apache/hadoop/mapred/HadoopMapRedUtil.scala |   17 +
 .../hadoop/mapreduce/HadoopMapReduceUtil.scala  |   17 +
 .../scala/spark/deploy/SparkHadoopUtil.scala    |   21 +
 .../apache/hadoop/mapred/HadoopMapRedUtil.scala |   17 +
 .../hadoop/mapreduce/HadoopMapReduceUtil.scala  |   17 +
 .../scala/spark/deploy/SparkHadoopUtil.scala    |   37 +-
 .../spark/deploy/yarn/ApplicationMaster.scala   |   92 +-
 .../yarn/ApplicationMasterArguments.scala       |   17 +
 .../scala/spark/deploy/yarn/Client.scala        |   75 +-
 .../spark/deploy/yarn/ClientArguments.scala     |   25 +-
 .../spark/deploy/yarn/WorkerRunnable.scala      |   56 +-
 .../deploy/yarn/YarnAllocationHandler.scala     |   17 +
 .../cluster/YarnClusterScheduler.scala          |   17 +
 .../apache/hadoop/mapred/HadoopMapRedUtil.scala |   17 +
 .../hadoop/mapreduce/HadoopMapReduceUtil.scala  |   17 +
 .../scala/spark/deploy/SparkHadoopUtil.scala    |   21 +
 .../java/spark/network/netty/FileClient.java    |   45 +-
 .../netty/FileClientChannelInitializer.java     |   17 +
 .../spark/network/netty/FileClientHandler.java  |   25 +
 .../java/spark/network/netty/FileServer.java    |   17 +
 .../netty/FileServerChannelInitializer.java     |   17 +
 .../spark/network/netty/FileServerHandler.java  |   17 +
 .../java/spark/network/netty/PathResolver.java  |   17 +
 .../deploy/static/bootstrap-responsive.min.css  |    9 -
 .../spark/deploy/static/bootstrap.min.css       |    9 -
 .../resources/spark/deploy/static/sorttable.js  |  495 ---------
 .../spark/deploy/static/spark_logo.png          |  Bin 14233 -> 0 bytes
 .../ui/static/bootstrap-responsive.min.css      |    9 +
 .../resources/spark/ui/static/bootstrap.min.css |    9 +
 .../main/resources/spark/ui/static/sorttable.js |  495 +++++++++
 .../spark/ui/static/spark-logo-77x50px-hd.png   |  Bin 0 -> 3536 bytes
 .../resources/spark/ui/static/spark_logo.png    |  Bin 0 -> 14233 bytes
 .../main/resources/spark/ui/static/webui.css    |   49 +
 core/src/main/scala/spark/Accumulators.scala    |   17 +
 core/src/main/scala/spark/Aggregator.scala      |   17 +
 .../scala/spark/BlockStoreShuffleFetcher.scala  |   18 +
 core/src/main/scala/spark/Cache.scala           |   17 +
 core/src/main/scala/spark/CacheManager.scala    |   17 +
 core/src/main/scala/spark/ClosureCleaner.scala  |   17 +
 core/src/main/scala/spark/Dependency.scala      |   17 +
 .../main/scala/spark/DoubleRDDFunctions.scala   |   17 +
 .../main/scala/spark/FetchFailedException.scala |   17 +
 core/src/main/scala/spark/HadoopWriter.scala    |   17 +
 core/src/main/scala/spark/HttpFileServer.scala  |   19 +-
 core/src/main/scala/spark/HttpServer.scala      |   17 +
 core/src/main/scala/spark/JavaSerializer.scala  |   17 +
 core/src/main/scala/spark/KryoSerializer.scala  |   21 +
 core/src/main/scala/spark/Logging.scala         |   17 +
 .../src/main/scala/spark/MapOutputTracker.scala |   17 +
 .../src/main/scala/spark/PairRDDFunctions.scala |   33 +-
 core/src/main/scala/spark/Partition.scala       |   17 +
 core/src/main/scala/spark/Partitioner.scala     |   17 +
 core/src/main/scala/spark/RDD.scala             |   84 +-
 .../main/scala/spark/RDDCheckpointData.scala    |   17 +
 .../scala/spark/SequenceFileRDDFunctions.scala  |   17 +
 .../main/scala/spark/SerializableWritable.scala |   17 +
 core/src/main/scala/spark/ShuffleFetcher.scala  |   17 +
 core/src/main/scala/spark/SizeEstimator.scala   |   17 +
 .../main/scala/spark/SoftReferenceCache.scala   |   17 +
 core/src/main/scala/spark/SparkContext.scala    |   61 +-
 core/src/main/scala/spark/SparkEnv.scala        |   20 +-
 core/src/main/scala/spark/SparkException.scala  |   17 +
 core/src/main/scala/spark/SparkFiles.java       |   17 +
 core/src/main/scala/spark/TaskContext.scala     |   17 +
 core/src/main/scala/spark/TaskEndReason.scala   |   21 +-
 core/src/main/scala/spark/TaskState.scala       |   17 +
 core/src/main/scala/spark/Utils.scala           |  184 +++-
 .../scala/spark/api/java/JavaDoubleRDD.scala    |   17 +
 .../main/scala/spark/api/java/JavaPairRDD.scala |   17 +
 .../src/main/scala/spark/api/java/JavaRDD.scala |   17 +
 .../main/scala/spark/api/java/JavaRDDLike.scala |   42 +
 .../scala/spark/api/java/JavaSparkContext.scala |   17 +
 .../java/JavaSparkContextVarargsWorkaround.java |   17 +
 .../scala/spark/api/java/StorageLevels.java     |   17 +
 .../java/function/DoubleFlatMapFunction.java    |   17 +
 .../spark/api/java/function/DoubleFunction.java |   17 +
 .../api/java/function/FlatMapFunction.scala     |   17 +
 .../api/java/function/FlatMapFunction2.scala    |   17 +
 .../scala/spark/api/java/function/Function.java |   17 +
 .../spark/api/java/function/Function2.java      |   17 +
 .../api/java/function/PairFlatMapFunction.java  |   17 +
 .../spark/api/java/function/PairFunction.java   |   17 +
 .../spark/api/java/function/VoidFunction.scala  |   19 +-
 .../api/java/function/WrappedFunction1.scala    |   17 +
 .../api/java/function/WrappedFunction2.scala    |   17 +
 .../spark/api/python/PythonPartitioner.scala    |   17 +
 .../main/scala/spark/api/python/PythonRDD.scala |   54 +-
 .../spark/api/python/PythonWorkerFactory.scala  |   37 +-
 .../spark/broadcast/BitTorrentBroadcast.scala   |   17 +
 .../main/scala/spark/broadcast/Broadcast.scala  |   17 +
 .../spark/broadcast/BroadcastFactory.scala      |   17 +
 .../scala/spark/broadcast/HttpBroadcast.scala   |   17 +
 .../scala/spark/broadcast/MultiTracker.scala    |   17 +
 .../main/scala/spark/broadcast/SourceInfo.scala |   17 +
 .../scala/spark/broadcast/TreeBroadcast.scala   |   17 +
 .../spark/deploy/ApplicationDescription.scala   |   17 +
 core/src/main/scala/spark/deploy/Command.scala  |   17 +
 .../main/scala/spark/deploy/DeployMessage.scala |   17 +
 .../main/scala/spark/deploy/ExecutorState.scala |   17 +
 .../main/scala/spark/deploy/JsonProtocol.scala  |  129 +--
 .../scala/spark/deploy/LocalSparkCluster.scala  |   18 +
 core/src/main/scala/spark/deploy/WebUI.scala    |   19 +-
 .../main/scala/spark/deploy/client/Client.scala |   17 +
 .../spark/deploy/client/ClientListener.scala    |   17 +
 .../scala/spark/deploy/client/TestClient.scala  |   17 +
 .../spark/deploy/client/TestExecutor.scala      |   17 +
 .../spark/deploy/master/ApplicationInfo.scala   |   17 +
 .../spark/deploy/master/ApplicationState.scala  |   17 +
 .../spark/deploy/master/ExecutorInfo.scala      |   17 +
 .../main/scala/spark/deploy/master/Master.scala |   39 +-
 .../spark/deploy/master/MasterArguments.scala   |   17 +
 .../scala/spark/deploy/master/MasterWebUI.scala |   74 --
 .../scala/spark/deploy/master/WorkerInfo.scala  |   17 +
 .../scala/spark/deploy/master/WorkerState.scala |   17 +
 .../deploy/master/ui/ApplicationPage.scala      |  117 ++
 .../spark/deploy/master/ui/IndexPage.scala      |  138 +++
 .../spark/deploy/master/ui/MasterWebUI.scala    |   76 ++
 .../spark/deploy/worker/ExecutorRunner.scala    |   49 +-
 .../main/scala/spark/deploy/worker/Worker.scala |   36 +-
 .../spark/deploy/worker/WorkerArguments.scala   |   17 +
 .../scala/spark/deploy/worker/WorkerWebUI.scala |   57 -
 .../spark/deploy/worker/ui/IndexPage.scala      |  117 ++
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |  182 ++++
 .../main/scala/spark/executor/Executor.scala    |   66 +-
 .../scala/spark/executor/ExecutorBackend.scala  |   17 +
 .../scala/spark/executor/ExecutorExitCode.scala |   17 +
 .../spark/executor/ExecutorURLClassLoader.scala |   17 +
 .../spark/executor/MesosExecutorBackend.scala   |   17 +
 .../executor/StandaloneExecutorBackend.scala    |   17 +
 .../main/scala/spark/executor/TaskMetrics.scala |   29 +-
 .../scala/spark/network/BufferMessage.scala     |   19 +-
 .../main/scala/spark/network/Connection.scala   |   17 +
 .../scala/spark/network/ConnectionManager.scala |   17 +
 .../spark/network/ConnectionManagerId.scala     |   17 +
 .../spark/network/ConnectionManagerTest.scala   |   17 +
 core/src/main/scala/spark/network/Message.scala |   17 +
 .../main/scala/spark/network/MessageChunk.scala |   17 +
 .../spark/network/MessageChunkHeader.scala      |   17 +
 .../main/scala/spark/network/ReceiverTest.scala |   17 +
 .../main/scala/spark/network/SenderTest.scala   |   17 +
 .../scala/spark/network/netty/FileHeader.scala  |   17 +
 .../spark/network/netty/ShuffleCopier.scala     |   82 +-
 .../spark/network/netty/ShuffleSender.scala     |   17 +
 core/src/main/scala/spark/package.scala         |   17 +
 .../partial/ApproximateActionListener.scala     |   17 +
 .../spark/partial/ApproximateEvaluator.scala    |   17 +
 .../scala/spark/partial/BoundedDouble.scala     |   17 +
 .../scala/spark/partial/CountEvaluator.scala    |   17 +
 .../spark/partial/GroupedCountEvaluator.scala   |   17 +
 .../spark/partial/GroupedMeanEvaluator.scala    |   17 +
 .../spark/partial/GroupedSumEvaluator.scala     |   17 +
 .../scala/spark/partial/MeanEvaluator.scala     |   17 +
 .../scala/spark/partial/PartialResult.scala     |   17 +
 .../scala/spark/partial/StudentTCacher.scala    |   17 +
 .../main/scala/spark/partial/SumEvaluator.scala |   17 +
 core/src/main/scala/spark/rdd/BlockRDD.scala    |   17 +
 .../src/main/scala/spark/rdd/CartesianRDD.scala |   17 +
 .../main/scala/spark/rdd/CheckpointRDD.scala    |   17 +
 .../src/main/scala/spark/rdd/CoGroupedRDD.scala |   17 +
 .../src/main/scala/spark/rdd/CoalescedRDD.scala |   17 +
 core/src/main/scala/spark/rdd/EmptyRDD.scala    |   17 +
 core/src/main/scala/spark/rdd/FilteredRDD.scala |   17 +
 .../main/scala/spark/rdd/FlatMappedRDD.scala    |   17 +
 core/src/main/scala/spark/rdd/GlommedRDD.scala  |   17 +
 core/src/main/scala/spark/rdd/HadoopRDD.scala   |   19 +
 core/src/main/scala/spark/rdd/JdbcRDD.scala     |   17 +
 .../main/scala/spark/rdd/MapPartitionsRDD.scala |   17 +
 .../spark/rdd/MapPartitionsWithIndexRDD.scala   |   17 +
 core/src/main/scala/spark/rdd/MappedRDD.scala   |   17 +
 .../src/main/scala/spark/rdd/NewHadoopRDD.scala |   17 +
 .../scala/spark/rdd/ParallelCollectionRDD.scala |   17 +
 .../scala/spark/rdd/PartitionPruningRDD.scala   |   17 +
 core/src/main/scala/spark/rdd/PipedRDD.scala    |   17 +
 core/src/main/scala/spark/rdd/SampledRDD.scala  |   17 +
 core/src/main/scala/spark/rdd/ShuffledRDD.scala |   17 +
 .../main/scala/spark/rdd/SubtractedRDD.scala    |   17 +
 core/src/main/scala/spark/rdd/UnionRDD.scala    |   17 +
 .../scala/spark/rdd/ZippedPartitionsRDD.scala   |   17 +
 core/src/main/scala/spark/rdd/ZippedRDD.scala   |   17 +
 .../main/scala/spark/scheduler/ActiveJob.scala  |   17 +
 .../scala/spark/scheduler/DAGScheduler.scala    |   75 +-
 .../spark/scheduler/DAGSchedulerEvent.scala     |   17 +
 .../scala/spark/scheduler/InputFormatInfo.scala |   21 +
 .../scala/spark/scheduler/JobListener.scala     |   17 +
 .../main/scala/spark/scheduler/JobLogger.scala  |  323 ++++++
 .../main/scala/spark/scheduler/JobResult.scala  |   19 +-
 .../main/scala/spark/scheduler/JobWaiter.scala  |   19 +-
 .../main/scala/spark/scheduler/MapStatus.scala  |   17 +
 .../main/scala/spark/scheduler/ResultTask.scala |   17 +
 .../scala/spark/scheduler/ShuffleMapTask.scala  |   17 +
 .../scala/spark/scheduler/SparkListener.scala   |   68 +-
 .../main/scala/spark/scheduler/SplitInfo.scala  |   17 +
 core/src/main/scala/spark/scheduler/Stage.scala |   23 +-
 .../main/scala/spark/scheduler/StageInfo.scala  |   19 +-
 core/src/main/scala/spark/scheduler/Task.scala  |   17 +
 .../main/scala/spark/scheduler/TaskResult.scala |   17 +
 .../scala/spark/scheduler/TaskScheduler.scala   |   17 +
 .../spark/scheduler/TaskSchedulerListener.scala |   17 +
 .../main/scala/spark/scheduler/TaskSet.scala    |   17 +
 .../scheduler/cluster/ClusterScheduler.scala    |   17 +
 .../cluster/ClusterTaskSetManager.scala         |   28 +-
 .../scheduler/cluster/ExecutorLossReason.scala  |   17 +
 .../scala/spark/scheduler/cluster/Pool.scala    |   17 +
 .../spark/scheduler/cluster/Schedulable.scala   |   17 +
 .../scheduler/cluster/SchedulableBuilder.scala  |   17 +
 .../scheduler/cluster/SchedulerBackend.scala    |   28 +-
 .../scheduler/cluster/SchedulingAlgorithm.scala |   17 +
 .../scheduler/cluster/SchedulingMode.scala      |   17 +
 .../cluster/SparkDeploySchedulerBackend.scala   |   17 +
 .../cluster/StandaloneClusterMessage.scala      |   17 +
 .../cluster/StandaloneSchedulerBackend.scala    |   17 +
 .../scheduler/cluster/TaskDescription.scala     |   17 +
 .../spark/scheduler/cluster/TaskInfo.scala      |   17 +
 .../scheduler/cluster/TaskSetManager.scala      |   17 +
 .../spark/scheduler/cluster/WorkerOffer.scala   |   17 +
 .../spark/scheduler/local/LocalScheduler.scala  |   34 +-
 .../scheduler/local/LocalTaskSetManager.scala   |   18 +
 .../mesos/CoarseMesosSchedulerBackend.scala     |   17 +
 .../scheduler/mesos/MesosSchedulerBackend.scala |  170 ++-
 .../scala/spark/serializer/Serializer.scala     |   17 +
 .../spark/serializer/SerializerManager.scala    |   17 +
 .../scala/spark/storage/BlockException.scala    |   17 +
 .../scala/spark/storage/BlockFetchTracker.scala |   17 +
 .../spark/storage/BlockFetcherIterator.scala    |   73 +-
 .../main/scala/spark/storage/BlockManager.scala |   17 +
 .../scala/spark/storage/BlockManagerId.scala    |   17 +
 .../spark/storage/BlockManagerMaster.scala      |   17 +
 .../spark/storage/BlockManagerMasterActor.scala |   17 +
 .../spark/storage/BlockManagerMessages.scala    |   17 +
 .../spark/storage/BlockManagerSlaveActor.scala  |   17 +
 .../scala/spark/storage/BlockManagerUI.scala    |   79 --
 .../spark/storage/BlockManagerWorker.scala      |   17 +
 .../main/scala/spark/storage/BlockMessage.scala |   17 +
 .../scala/spark/storage/BlockMessageArray.scala |   17 +
 .../scala/spark/storage/BlockObjectWriter.scala |   17 +
 .../main/scala/spark/storage/BlockStore.scala   |   17 +
 .../main/scala/spark/storage/DiskStore.scala    |   70 +-
 .../main/scala/spark/storage/MemoryStore.scala  |   17 +
 .../main/scala/spark/storage/PutResult.scala    |   17 +
 .../spark/storage/ShuffleBlockManager.scala     |   19 +-
 .../main/scala/spark/storage/StorageLevel.scala |   17 +
 .../main/scala/spark/storage/StorageUtils.scala |   28 +-
 .../scala/spark/storage/ThreadingTest.scala     |   17 +
 core/src/main/scala/spark/ui/JettyUtils.scala   |  132 +++
 core/src/main/scala/spark/ui/Page.scala         |   20 +
 core/src/main/scala/spark/ui/SparkUI.scala      |   79 ++
 core/src/main/scala/spark/ui/UIUtils.scala      |  135 +++
 .../scala/spark/ui/UIWorkloadGenerator.scala    |   88 ++
 .../main/scala/spark/ui/env/EnvironmentUI.scala |   86 ++
 .../main/scala/spark/ui/jobs/IndexPage.scala    |  129 +++
 .../scala/spark/ui/jobs/JobProgressUI.scala     |  144 +++
 .../main/scala/spark/ui/jobs/StagePage.scala    |  131 +++
 .../scala/spark/ui/storage/BlockManagerUI.scala |   41 +
 .../main/scala/spark/ui/storage/IndexPage.scala |   81 ++
 .../main/scala/spark/ui/storage/RDDPage.scala   |  130 +++
 core/src/main/scala/spark/util/AkkaUtils.scala  |   59 +-
 .../scala/spark/util/BoundedPriorityQueue.scala |   17 +
 .../spark/util/ByteBufferInputStream.scala      |   17 +
 .../scala/spark/util/CompletionIterator.scala   |   19 +-
 .../main/scala/spark/util/Distribution.scala    |   19 +-
 .../src/main/scala/spark/util/IdGenerator.scala |   17 +
 core/src/main/scala/spark/util/IntParam.scala   |   17 +
 .../src/main/scala/spark/util/MemoryParam.scala |   17 +
 .../main/scala/spark/util/MetadataCleaner.scala |   17 +
 .../main/scala/spark/util/NextIterator.scala    |   19 +-
 .../spark/util/RateLimitedOutputStream.scala    |   17 +
 .../scala/spark/util/SerializableBuffer.scala   |   17 +
 .../src/main/scala/spark/util/StatCounter.scala |   17 +
 .../scala/spark/util/TimeStampedHashMap.scala   |   17 +
 .../scala/spark/util/TimeStampedHashSet.scala   |   17 +
 core/src/main/scala/spark/util/Vector.scala     |   17 +
 .../main/twirl/spark/common/layout.scala.html   |   35 -
 .../spark/deploy/master/app_details.scala.html  |   38 -
 .../spark/deploy/master/app_row.scala.html      |   20 -
 .../spark/deploy/master/app_table.scala.html    |   21 -
 .../spark/deploy/master/executor_row.scala.html |   15 -
 .../deploy/master/executors_table.scala.html    |   19 -
 .../twirl/spark/deploy/master/index.scala.html  |   53 -
 .../spark/deploy/master/worker_row.scala.html   |   14 -
 .../spark/deploy/master/worker_table.scala.html |   18 -
 .../spark/deploy/worker/executor_row.scala.html |   20 -
 .../deploy/worker/executors_table.scala.html    |   18 -
 .../twirl/spark/deploy/worker/index.scala.html  |   44 -
 .../main/twirl/spark/storage/index.scala.html   |   40 -
 .../src/main/twirl/spark/storage/rdd.scala.html |   81 --
 .../twirl/spark/storage/rdd_table.scala.html    |   32 -
 .../twirl/spark/storage/worker_table.scala.html |   24 -
 core/src/test/resources/fairscheduler.xml       |   18 +
 core/src/test/resources/log4j.properties        |   20 +-
 .../src/test/scala/spark/AccumulatorSuite.scala |   17 +
 core/src/test/scala/spark/BroadcastSuite.scala  |   17 +
 core/src/test/scala/spark/CheckpointSuite.scala |   27 +
 .../test/scala/spark/ClosureCleanerSuite.scala  |   17 +
 .../src/test/scala/spark/DistributedSuite.scala |   19 +
 core/src/test/scala/spark/DriverSuite.scala     |   21 +
 core/src/test/scala/spark/FailureSuite.scala    |   17 +
 core/src/test/scala/spark/FileServerSuite.scala |   32 +-
 core/src/test/scala/spark/FileSuite.scala       |   17 +
 core/src/test/scala/spark/JavaAPISuite.java     |   17 +
 .../test/scala/spark/KryoSerializerSuite.scala  |   17 +
 .../test/scala/spark/LocalSparkContext.scala    |   28 +-
 .../scala/spark/MapOutputTrackerSuite.scala     |   17 +
 .../scala/spark/PairRDDFunctionsSuite.scala     |  304 ++++++
 .../test/scala/spark/PartitioningSuite.scala    |   36 +-
 core/src/test/scala/spark/PipedRDDSuite.scala   |   35 +-
 core/src/test/scala/spark/RDDSuite.scala        |  163 ++-
 .../test/scala/spark/SharedSparkContext.scala   |   42 +
 .../test/scala/spark/ShuffleNettySuite.scala    |   17 +
 core/src/test/scala/spark/ShuffleSuite.scala    |  316 +-----
 .../test/scala/spark/SizeEstimatorSuite.scala   |   89 +-
 core/src/test/scala/spark/SortingSuite.scala    |   40 +-
 core/src/test/scala/spark/ThreadingSuite.scala  |   17 +
 core/src/test/scala/spark/UnpersistSuite.scala  |   47 +
 core/src/test/scala/spark/UtilsSuite.scala      |  119 ++-
 .../scala/spark/ZippedPartitionsSuite.scala     |   20 +-
 .../src/test/scala/spark/rdd/JdbcRDDSuite.scala |   17 +
 .../rdd/ParallelCollectionSplitSuite.scala      |   17 +
 .../spark/scheduler/ClusterSchedulerSuite.scala |   17 +
 .../spark/scheduler/DAGSchedulerSuite.scala     |   17 +
 .../scala/spark/scheduler/JobLoggerSuite.scala  |  121 +++
 .../spark/scheduler/LocalSchedulerSuite.scala   |   17 +
 .../spark/scheduler/SparkListenerSuite.scala    |   19 +-
 .../spark/scheduler/TaskContextSuite.scala      |   17 +
 .../scala/spark/storage/BlockManagerSuite.scala |   17 +
 core/src/test/scala/spark/ui/UISuite.scala      |   46 +
 .../scala/spark/util/DistributionSuite.scala    |   17 +
 .../scala/spark/util/NextIteratorSuite.scala    |   17 +
 .../util/RateLimitedOutputStreamSuite.scala     |   17 +
 docs/_plugins/copy_api_dirs.rb                  |   17 +
 docs/configuration.md                           |   65 +-
 docs/ec2-scripts.md                             |    5 +-
 docs/python-programming-guide.md                |   12 +-
 docs/scala-programming-guide.md                 |   10 +-
 docs/streaming-programming-guide.md             |    4 +-
 docs/tuning.md                                  |    6 +-
 ec2/spark-ec2                                   |    2 +
 ec2/spark_ec2.py                                |    2 +
 examples/pom.xml                                |   26 +
 .../main/java/spark/examples/JavaHdfsLR.java    |   17 +
 .../main/java/spark/examples/JavaKMeans.java    |   17 +
 .../main/java/spark/examples/JavaLogQuery.java  |   17 +
 .../main/java/spark/examples/JavaSparkPi.java   |   17 +
 .../src/main/java/spark/examples/JavaTC.java    |   17 +
 .../main/java/spark/examples/JavaWordCount.java |   17 +
 .../streaming/examples/JavaFlumeEventCount.java |   17 +
 .../examples/JavaNetworkWordCount.java          |   17 +
 .../streaming/examples/JavaQueueStream.java     |   17 +
 .../scala/spark/examples/BroadcastTest.scala    |   17 +
 .../scala/spark/examples/CassandraTest.scala    |   17 +
 .../spark/examples/ExceptionHandlingTest.scala  |   17 +
 .../main/scala/spark/examples/GroupByTest.scala |   17 +
 .../main/scala/spark/examples/HBaseTest.scala   |   19 +-
 .../main/scala/spark/examples/HdfsTest.scala    |   17 +
 .../main/scala/spark/examples/LocalALS.scala    |   17 +
 .../main/scala/spark/examples/LocalFileLR.scala |   17 +
 .../main/scala/spark/examples/LocalKMeans.scala |   17 +
 .../src/main/scala/spark/examples/LocalLR.scala |   17 +
 .../src/main/scala/spark/examples/LocalPi.scala |   17 +
 .../main/scala/spark/examples/LogQuery.scala    |   17 +
 .../spark/examples/MultiBroadcastTest.scala     |   17 +
 .../examples/SimpleSkewedGroupByTest.scala      |   17 +
 .../spark/examples/SkewedGroupByTest.scala      |   17 +
 .../main/scala/spark/examples/SparkALS.scala    |   17 +
 .../main/scala/spark/examples/SparkHdfsLR.scala |   17 +
 .../main/scala/spark/examples/SparkKMeans.scala |   17 +
 .../src/main/scala/spark/examples/SparkLR.scala |   17 +
 .../src/main/scala/spark/examples/SparkPi.scala |   17 +
 .../src/main/scala/spark/examples/SparkTC.scala |   17 +
 .../streaming/examples/ActorWordCount.scala     |   17 +
 .../streaming/examples/FlumeEventCount.scala    |   17 +
 .../streaming/examples/HdfsWordCount.scala      |   17 +
 .../streaming/examples/KafkaWordCount.scala     |   19 +-
 .../streaming/examples/NetworkWordCount.scala   |   17 +
 .../spark/streaming/examples/QueueStream.scala  |   17 +
 .../streaming/examples/RawNetworkGrep.scala     |   17 +
 .../examples/StatefulNetworkWordCount.scala     |   67 ++
 .../streaming/examples/TwitterAlgebirdCMS.scala |   26 +-
 .../streaming/examples/TwitterAlgebirdHLL.scala |   26 +-
 .../streaming/examples/TwitterPopularTags.scala |   26 +-
 .../streaming/examples/ZeroMQWordCount.scala    |   17 +
 .../clickstream/PageViewGenerator.scala         |   17 +
 .../examples/clickstream/PageViewStream.scala   |   17 +
 make-distribution.sh                            |   58 +
 mllib/data/als/test.data                        |   16 +
 mllib/data/lr-data/random.data                  | 1000 ++++++++++++++++++
 mllib/data/ridge-data/lpsa.data                 |   67 ++
 .../scala/spark/mllib/clustering/KMeans.scala   |  334 ++++++
 .../spark/mllib/clustering/KMeansModel.scala    |   44 +
 .../spark/mllib/clustering/LocalKMeans.scala    |  105 ++
 .../spark/mllib/optimization/Gradient.scala     |   50 +
 .../mllib/optimization/GradientDescent.scala    |   79 ++
 .../spark/mllib/optimization/Updater.scala      |   44 +
 .../scala/spark/mllib/recommendation/ALS.scala  |  436 ++++++++
 .../MatrixFactorizationModel.scala              |   40 +
 .../mllib/regression/LogisticRegression.scala   |  175 +++
 .../LogisticRegressionGenerator.scala           |   58 +
 .../spark/mllib/regression/Regression.scala     |   38 +
 .../mllib/regression/RidgeRegression.scala      |  211 ++++
 .../regression/RidgeRegressionGenerator.scala   |   72 ++
 .../main/scala/spark/mllib/util/MLUtils.scala   |  112 ++
 mllib/src/test/resources/log4j.properties       |   28 +
 .../spark/mllib/clustering/KMeansSuite.scala    |  170 +++
 .../spark/mllib/recommendation/ALSSuite.scala   |   97 ++
 .../regression/LogisticRegressionSuite.scala    |   74 ++
 .../mllib/regression/RidgeRegressionSuite.scala |   64 ++
 pom.xml                                         |  103 +-
 project/SparkBuild.scala                        |   47 +-
 project/build.properties                        |   17 +
 project/plugins.sbt                             |    2 -
 pyspark                                         |   17 +
 python/epydoc.conf                              |   17 +
 python/examples/als.py                          |   17 +
 python/examples/kmeans.py                       |   17 +
 python/examples/logistic_regression.py          |   17 +
 python/examples/pi.py                           |   17 +
 python/examples/transitive_closure.py           |   17 +
 python/examples/wordcount.py                    |   17 +
 python/pyspark/accumulators.py                  |   17 +
 python/pyspark/broadcast.py                     |   17 +
 python/pyspark/context.py                       |   17 +
 python/pyspark/daemon.py                        |   59 +-
 python/pyspark/files.py                         |   17 +
 python/pyspark/java_gateway.py                  |   17 +
 python/pyspark/rdd.py                           |   17 +
 python/pyspark/serializers.py                   |   17 +
 python/pyspark/shell.py                         |   17 +
 python/pyspark/tests.py                         |   17 +
 python/pyspark/worker.py                        |   17 +
 python/run-tests                                |   20 +-
 repl-bin/pom.xml                                |   17 +
 repl-bin/src/deb/bin/run                        |   19 +-
 repl-bin/src/deb/bin/spark-executor             |   17 +
 repl-bin/src/deb/bin/spark-shell                |   17 +
 repl/pom.xml                                    |   17 +
 .../scala/spark/repl/ExecutorClassLoader.scala  |   17 +
 repl/src/main/scala/spark/repl/Main.scala       |   17 +
 repl/src/main/scala/spark/repl/SparkILoop.scala |   11 +-
 repl/src/test/resources/log4j.properties        |   19 +-
 repl/src/test/scala/spark/repl/ReplSuite.scala  |   55 +-
 run                                             |  151 ++-
 run.cmd                                         |   18 +
 run2.cmd                                        |   63 +-
 sbt/sbt                                         |   22 +-
 sbt/sbt.cmd                                     |   20 +
 spark-executor                                  |   18 +
 spark-shell                                     |   86 +-
 spark-shell.cmd                                 |   18 +
 streaming/pom.xml                               |   21 +
 .../main/scala/spark/streaming/Checkpoint.scala |   26 +-
 .../main/scala/spark/streaming/DStream.scala    |   26 +-
 .../spark/streaming/DStreamCheckpointData.scala |   17 +
 .../scala/spark/streaming/DStreamGraph.scala    |   17 +
 .../main/scala/spark/streaming/Duration.scala   |   23 +-
 .../main/scala/spark/streaming/Interval.scala   |   17 +
 .../src/main/scala/spark/streaming/Job.scala    |   17 +
 .../main/scala/spark/streaming/JobManager.scala |   17 +
 .../spark/streaming/NetworkInputTracker.scala   |   17 +
 .../spark/streaming/PairDStreamFunctions.scala  |   20 +-
 .../main/scala/spark/streaming/Scheduler.scala  |   17 +
 .../spark/streaming/StreamingContext.scala      |   77 +-
 .../src/main/scala/spark/streaming/Time.scala   |   19 +-
 .../spark/streaming/api/java/JavaDStream.scala  |   19 +-
 .../streaming/api/java/JavaDStreamLike.scala    |   19 +-
 .../streaming/api/java/JavaPairDStream.scala    |   17 +
 .../api/java/JavaStreamingContext.scala         |  140 ++-
 .../streaming/dstream/CoGroupedDStream.scala    |   17 +
 .../dstream/ConstantInputDStream.scala          |   19 +-
 .../streaming/dstream/FileInputDStream.scala    |   17 +
 .../streaming/dstream/FilteredDStream.scala     |   17 +
 .../dstream/FlatMapValuedDStream.scala          |   17 +
 .../streaming/dstream/FlatMappedDStream.scala   |   17 +
 .../streaming/dstream/FlumeInputDStream.scala   |   17 +
 .../streaming/dstream/ForEachDStream.scala      |   17 +
 .../streaming/dstream/GlommedDStream.scala      |   17 +
 .../spark/streaming/dstream/InputDStream.scala  |   17 +
 .../streaming/dstream/KafkaInputDStream.scala   |  111 +-
 .../dstream/MapPartitionedDStream.scala         |   17 +
 .../streaming/dstream/MapValuedDStream.scala    |   17 +
 .../spark/streaming/dstream/MappedDStream.scala |   17 +
 .../streaming/dstream/NetworkInputDStream.scala |   37 +-
 .../dstream/PluggableInputDStream.scala         |   17 +
 .../streaming/dstream/QueueInputDStream.scala   |   17 +
 .../streaming/dstream/RawInputDStream.scala     |   17 +
 .../dstream/ReducedWindowedDStream.scala        |   17 +
 .../streaming/dstream/ShuffledDStream.scala     |   17 +
 .../streaming/dstream/SocketInputDStream.scala  |   17 +
 .../spark/streaming/dstream/StateDStream.scala  |   17 +
 .../streaming/dstream/TransformedDStream.scala  |   17 +
 .../streaming/dstream/TwitterInputDStream.scala |   47 +-
 .../spark/streaming/dstream/UnionDStream.scala  |   17 +
 .../streaming/dstream/WindowedDStream.scala     |   17 +
 .../streaming/receivers/ActorReceiver.scala     |   24 +-
 .../streaming/receivers/ZeroMQReceiver.scala    |   17 +
 .../main/scala/spark/streaming/util/Clock.scala |   17 +
 .../streaming/util/MasterFailureTest.scala      |   17 +
 .../spark/streaming/util/RawTextHelper.scala    |   17 +
 .../spark/streaming/util/RawTextSender.scala    |   17 +
 .../spark/streaming/util/RecurringTimer.scala   |   17 +
 .../test/java/spark/streaming/JavaAPISuite.java |   31 +-
 .../java/spark/streaming/JavaTestUtils.scala    |   19 +-
 streaming/src/test/resources/log4j.properties   |   19 +-
 .../spark/streaming/BasicOperationsSuite.scala  |   21 +-
 .../scala/spark/streaming/CheckpointSuite.scala |   17 +
 .../scala/spark/streaming/FailureSuite.scala    |   17 +
 .../spark/streaming/InputStreamsSuite.scala     |   28 +
 .../scala/spark/streaming/TestSuiteBase.scala   |   17 +
 .../spark/streaming/WindowOperationsSuite.scala |   17 +
 534 files changed, 16388 insertions(+), 2690 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/87d586e4/README.md
----------------------------------------------------------------------


[10/50] [abbrv] git commit: Merge pull request #698 from Reinvigorate/sm-deps-change

Posted by ma...@apache.org.
Merge pull request #698 from Reinvigorate/sm-deps-change

changing com.google.code.findbugs maven coordinates

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/89e8549f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/89e8549f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/89e8549f

Branch: refs/heads/master
Commit: 89e8549f539f81b88e7bff93c898620d7aad0b63
Parents: 1889f9f c4d5b01
Author: Matei Zaharia <ma...@gmail.com>
Authored: Sat Jul 13 16:11:08 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Sat Jul 13 16:11:08 2013 -0700

----------------------------------------------------------------------
 project/SparkBuild.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[38/50] [abbrv] git commit: SPARK-814: Result stages should be named after action

Posted by ma...@apache.org.
SPARK-814: Result stages should be named after action


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/5c388808
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/5c388808
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/5c388808

Branch: refs/heads/master
Commit: 5c388808a81ebfb1fc23511882c18f9ae76ec509
Parents: f347cc3
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Tue Jul 16 11:01:14 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Tue Jul 16 11:01:14 2013 -0700

----------------------------------------------------------------------
 .../scala/spark/scheduler/DAGScheduler.scala    | 22 +++++++++++++-------
 core/src/main/scala/spark/scheduler/Stage.scala |  5 +++--
 .../main/scala/spark/ui/jobs/IndexPage.scala    |  2 +-
 .../scala/spark/scheduler/JobLoggerSuite.scala  |  4 ++--
 4 files changed, 20 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5c388808/core/src/main/scala/spark/scheduler/DAGScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
index 3d3b9ea..30a648f 100644
--- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
@@ -150,7 +150,13 @@ class DAGScheduler(
    * as a result stage for the final RDD used directly in an action. The stage will also be given
    * the provided priority.
    */
-  private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = {
+  private def newStage(
+      rdd: RDD[_],
+      shuffleDep: Option[ShuffleDependency[_,_]],
+      priority: Int,
+      callSite: Option[String] = None)
+    : Stage =
+  {
     if (shuffleDep != None) {
       // Kind of ugly: need to register RDDs with the cache and map output tracker here
       // since we can't do it in the RDD constructor because # of partitions is unknown
@@ -158,7 +164,7 @@ class DAGScheduler(
       mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size)
     }
     val id = nextStageId.getAndIncrement()
-    val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority)
+    val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority, callSite)
     idToStage(id) = stage
     stageToInfos(stage) = StageInfo(stage)
     stage
@@ -286,12 +292,12 @@ class DAGScheduler(
     event match {
       case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) =>
         val runId = nextRunId.getAndIncrement()
-        val finalStage = newStage(finalRDD, None, runId)
+        val finalStage = newStage(finalRDD, None, runId, Some(callSite))
         val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties)
         clearCacheLocs()
         logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length +
                 " output partitions (allowLocal=" + allowLocal + ")")
-        logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")")
+        logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")")
         logInfo("Parents of final stage: " + finalStage.parents)
         logInfo("Missing parents: " + getMissingParentStages(finalStage))
         if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) {
@@ -502,7 +508,7 @@ class DAGScheduler(
         case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0)
         case _ => "Unkown"
       }
-      logInfo("%s (%s) finished in %s s".format(stage, stage.origin, serviceTime))
+      logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime))
       stage.completionTime = Some(System.currentTimeMillis)
       val stageComp = StageCompleted(stageToInfos(stage))
       sparkListeners.foreach{_.onStageCompleted(stageComp)}
@@ -568,7 +574,7 @@ class DAGScheduler(
               if (stage.outputLocs.count(_ == Nil) != 0) {
                 // Some tasks had failed; let's resubmit this stage
                 // TODO: Lower-level scheduler should also deal with this
-                logInfo("Resubmitting " + stage + " (" + stage.origin +
+                logInfo("Resubmitting " + stage + " (" + stage.name +
                   ") because some of its tasks had failed: " +
                   stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", "))
                 submitStage(stage)
@@ -600,7 +606,7 @@ class DAGScheduler(
         running -= failedStage
         failed += failedStage
         // TODO: Cancel running tasks in the stage
-        logInfo("Marking " + failedStage + " (" + failedStage.origin +
+        logInfo("Marking " + failedStage + " (" + failedStage.name +
           ") for resubmision due to a fetch failure")
         // Mark the map whose fetch failed as broken in the map stage
         val mapStage = shuffleToMapStage(shuffleId)
@@ -608,7 +614,7 @@ class DAGScheduler(
           mapStage.removeOutputLoc(mapId, bmAddress)
           mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress)
         }
-        logInfo("The failed fetch was from " + mapStage + " (" + mapStage.origin +
+        logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name +
           "); marking it for resubmission")
         failed += mapStage
         // Remember that a fetch failed now; this is used to resubmit the broken

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5c388808/core/src/main/scala/spark/scheduler/Stage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala
index 539cf82..4937eb3 100644
--- a/core/src/main/scala/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/spark/scheduler/Stage.scala
@@ -24,7 +24,8 @@ private[spark] class Stage(
     val rdd: RDD[_],
     val shuffleDep: Option[ShuffleDependency[_,_]],  // Output shuffle if stage is a map stage
     val parents: List[Stage],
-    val priority: Int)
+    val priority: Int,
+    callSite: Option[String])
   extends Logging {
 
   val isShuffleMap = shuffleDep != None
@@ -85,7 +86,7 @@ private[spark] class Stage(
     return id
   }
 
-  def origin: String = rdd.origin
+  val name = callSite.getOrElse(rdd.origin)
 
   override def toString = "Stage " + id
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5c388808/core/src/main/scala/spark/ui/jobs/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala
index 7907ab3..2df5f01 100644
--- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala
+++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala
@@ -89,7 +89,7 @@ private[spark] class IndexPage(parent: JobProgressUI) {
 
     <tr>
       <td>{s.id}</td>
-      <td><a href={"/stages/stage?id=%s".format(s.id)}>{s.origin}</a></td>
+      <td><a href={"/stages/stage?id=%s".format(s.id)}>{s.name}</a></td>
       <td>{submissionTime}</td>
       <td>{getElapsedTime(s.submissionTime,
              s.completionTime.getOrElse(System.currentTimeMillis()))}</td>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5c388808/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
index 699901f..2474d74 100644
--- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
@@ -37,8 +37,8 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
     val parentRdd = makeRdd(4, Nil)
     val shuffleDep = new ShuffleDependency(parentRdd, null)
     val rootRdd = makeRdd(4, List(shuffleDep))
-    val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) 
-    val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID)
+    val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID, None)
+    val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID, None)
     
     joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4))
     joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName)


[39/50] [abbrv] git commit: Merge branch 'master' of github.com:mesos/spark

Posted by ma...@apache.org.
Merge branch 'master' of github.com:mesos/spark


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/b1f9f647
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/b1f9f647
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/b1f9f647

Branch: refs/heads/master
Commit: b1f9f64743400651cd74fe5f7f04f5891ba6d1c5
Parents: 5c38880 8a8a8f2
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Tue Jul 16 11:01:53 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Tue Jul 16 11:01:53 2013 -0700

----------------------------------------------------------------------
 .../spark/deploy/yarn/ApplicationMaster.scala   | 58 ++++++++++++++------
 core/src/main/scala/spark/KryoSerializer.scala  |  4 ++
 .../spark/deploy/master/ui/IndexPage.scala      |  4 +-
 .../scala/spark/scheduler/DAGScheduler.scala    |  9 +++
 core/src/main/scala/spark/ui/UIUtils.scala      |  2 +-
 core/src/test/scala/spark/RDDSuite.scala        |  6 ++
 docs/configuration.md                           | 12 +++-
 .../scala/spark/mllib/recommendation/ALS.scala  | 27 ++++++---
 8 files changed, 93 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/b1f9f647/core/src/main/scala/spark/scheduler/DAGScheduler.scala
----------------------------------------------------------------------


[32/50] [abbrv] git commit: Removed log message, added field in master UI to link to log UI

Posted by ma...@apache.org.
Removed log message, added field in master UI to link to log UI


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/fbf5aa76
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/fbf5aa76
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/fbf5aa76

Branch: refs/heads/master
Commit: fbf5aa761e40649d0488a8673d488882c8bdae48
Parents: eac381a
Author: Karen Feng <ka...@gmail.com>
Authored: Mon Jul 15 15:50:03 2013 -0700
Committer: Karen Feng <ka...@gmail.com>
Committed: Mon Jul 15 15:50:03 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/deploy/master/Master.scala | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/fbf5aa76/core/src/main/scala/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala
index 87a7791..4dd6c44 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/spark/deploy/master/Master.scala
@@ -80,7 +80,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       logInfo("Registering app " + description.name)
       val app = addApplication(description, sender)
       logInfo("Registered app " + description.name + " with ID " + app.id)
-      logInfo("Started App web UI at " + description.appUiUrl)
       waitingApps += app
       context.watch(sender)  // This doesn't work with remote actors but helps for testing
       sender ! RegisteredApplication(app.id)


[42/50] [abbrv] Add Apache license headers and LICENSE and NOTICE files

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala
index 51c3c9f..33ab324 100644
--- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import spark.streaming._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
index 528778e..8770abd 100644
--- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import spark.streaming.{Seconds, StreamingContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala
index 896e9fd..cba5c98 100644
--- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala
+++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import spark.streaming.{Seconds, StreamingContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
index 65f0b6d..682b99f 100644
--- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
+++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import spark.streaming.{Seconds, StreamingContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala
index 74d0d33..e264fae 100644
--- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import akka.actor.ActorSystem

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
index 4c6e08b..375d5c9 100644
--- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples.clickstream
 
 import java.net.{InetAddress,ServerSocket,Socket,SocketException}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
index e226a4a..a24342b 100644
--- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples.clickstream
 
 import spark.streaming.{Seconds, StreamingContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/make-distribution.sh
----------------------------------------------------------------------
diff --git a/make-distribution.sh b/make-distribution.sh
index feb13d5..0116215 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -1,4 +1,22 @@
 #!/bin/bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 #
 # Script to create a binary distribution for easy deploys of Spark.
 # The distribution directory defaults to dist/ but can be overridden below.
@@ -12,6 +30,7 @@
 # 3) Verify master is up by visiting web page, ie http://master-ip:8080.  Note the spark:// URL.
 # 4) ./bin/start-slave.sh 1 <<spark:// URL>>
 # 5) MASTER="spark://my-master-ip:7077" ./spark-shell
+#
 
 # Figure out where the Spark framework is installed
 FWDIR="$(cd `dirname $0`; pwd)"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala
index b0e141f..d875d6d 100644
--- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala
+++ b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.clustering
 
 import scala.collection.mutable.ArrayBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala
index 4fd0646..b8f80e8 100644
--- a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala
+++ b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.clustering
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala
index e12b3be..89fe7d7 100644
--- a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala
+++ b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.clustering
 
 import scala.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala
index 90b0999..2fb0c81 100644
--- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala
+++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.optimization
 
 import org.jblas.DoubleMatrix

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala
index eff853f..e1b73bc 100644
--- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala
+++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.optimization
 
 import spark.{Logging, RDD, SparkContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/optimization/Updater.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala
index ea80bfc..b864fd4 100644
--- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala
+++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.optimization
 
 import org.jblas.DoubleMatrix

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
index 4c18cbd..7da9639 100644
--- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.recommendation
 
 import scala.collection.mutable.{ArrayBuffer, BitSet}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala
index fb812a6..38637b3 100644
--- a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala
+++ b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.recommendation
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala
index e4db7bb..bb294c2 100644
--- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala
+++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.regression
 
 import spark.{Logging, RDD, SparkContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala
index 6e7c023..8094d22 100644
--- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala
+++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.regression
 
 import scala.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/regression/Regression.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala
index f79974c..645204d 100644
--- a/mllib/src/main/scala/spark/mllib/regression/Regression.scala
+++ b/mllib/src/main/scala/spark/mllib/regression/Regression.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.regression
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala
index 5f813df..7c7f912 100644
--- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala
+++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.regression
 
 import spark.{Logging, RDD, SparkContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala
index b83f505..c2260ae 100644
--- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala
+++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.regression
 
 import scala.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/main/scala/spark/mllib/util/MLUtils.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala
index 08a031d..b5e564d 100644
--- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala
+++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.util
 
 import spark.{RDD, SparkContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties
index 390c927..a112e0b 100644
--- a/mllib/src/test/resources/log4j.properties
+++ b/mllib/src/test/resources/log4j.properties
@@ -1,4 +1,21 @@
-# Set everything to be logged to the file core/target/unit-tests.log 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file core/target/unit-tests.log
 log4j.rootCategory=INFO, file
 log4j.appender.file=org.apache.log4j.FileAppender
 log4j.appender.file.append=false

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala
index cb096f3..bebade9 100644
--- a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala
+++ b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.clustering
 
 import scala.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala
index 2ada9ae..f98590b 100644
--- a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.recommendation
 
 import scala.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala
index 04d3400..bc9bfd0 100644
--- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.regression
 
 import scala.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala
----------------------------------------------------------------------
diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala
index df41dbb..3c588c6 100644
--- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala
+++ b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.mllib.regression
 
 import scala.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6fa1f1d..4b48072 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
 <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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 178a056..7a4d4c4 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -1,3 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 
 import sbt._
 import sbt.Classpaths.publishTask

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/project/build.properties
----------------------------------------------------------------------
diff --git a/project/build.properties b/project/build.properties
index 9b860e2..08e1713 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -1 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 sbt.version=0.12.3

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/pyspark
----------------------------------------------------------------------
diff --git a/pyspark b/pyspark
index d662e90..37a3554 100755
--- a/pyspark
+++ b/pyspark
@@ -1,5 +1,22 @@
 #!/usr/bin/env bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Figure out where the Scala framework is installed
 FWDIR="$(cd `dirname $0`; pwd)"
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/epydoc.conf
----------------------------------------------------------------------
diff --git a/python/epydoc.conf b/python/epydoc.conf
index 45102cd..d5d5aa5 100644
--- a/python/epydoc.conf
+++ b/python/epydoc.conf
@@ -1,5 +1,22 @@
 [epydoc] # Epydoc section marker (required by ConfigParser)
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Information about the project.
 name: PySpark
 url: http://spark-project.org

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/examples/als.py
----------------------------------------------------------------------
diff --git a/python/examples/als.py b/python/examples/als.py
index 010f800..f2b2eee 100755
--- a/python/examples/als.py
+++ b/python/examples/als.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 """
 This example requires numpy (http://www.numpy.org/)
 """

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/examples/kmeans.py
----------------------------------------------------------------------
diff --git a/python/examples/kmeans.py b/python/examples/kmeans.py
index 72cf9f8..c670556 100644
--- a/python/examples/kmeans.py
+++ b/python/examples/kmeans.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 """
 This example requires numpy (http://www.numpy.org/)
 """

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/examples/logistic_regression.py
----------------------------------------------------------------------
diff --git a/python/examples/logistic_regression.py b/python/examples/logistic_regression.py
index f13698a..54d227d 100755
--- a/python/examples/logistic_regression.py
+++ b/python/examples/logistic_regression.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 """
 This example requires numpy (http://www.numpy.org/)
 """

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/examples/pi.py
----------------------------------------------------------------------
diff --git a/python/examples/pi.py b/python/examples/pi.py
index 127cba0..33c026e 100644
--- a/python/examples/pi.py
+++ b/python/examples/pi.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 import sys
 from random import random
 from operator import add

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/examples/transitive_closure.py
----------------------------------------------------------------------
diff --git a/python/examples/transitive_closure.py b/python/examples/transitive_closure.py
index 7f85a10..40be3b5 100644
--- a/python/examples/transitive_closure.py
+++ b/python/examples/transitive_closure.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 import sys
 from random import Random
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/examples/wordcount.py
----------------------------------------------------------------------
diff --git a/python/examples/wordcount.py b/python/examples/wordcount.py
index 8571606..41c846b 100644
--- a/python/examples/wordcount.py
+++ b/python/examples/wordcount.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 import sys
 from operator import add
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/accumulators.py
----------------------------------------------------------------------
diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py
index 3e9d7d3..d367f91 100644
--- a/python/pyspark/accumulators.py
+++ b/python/pyspark/accumulators.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 """
 >>> from pyspark.context import SparkContext
 >>> sc = SparkContext('local', 'test')

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/broadcast.py
----------------------------------------------------------------------
diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py
index def810d..dfdaba2 100644
--- a/python/pyspark/broadcast.py
+++ b/python/pyspark/broadcast.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 """
 >>> from pyspark.context import SparkContext
 >>> sc = SparkContext('local', 'test')

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/context.py
----------------------------------------------------------------------
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 657fe6f..2f741cb 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 import os
 import shutil
 import sys

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/daemon.py
----------------------------------------------------------------------
diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py
index 78c9457..eb18ec0 100644
--- a/python/pyspark/daemon.py
+++ b/python/pyspark/daemon.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 import os
 import signal
 import socket

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/files.py
----------------------------------------------------------------------
diff --git a/python/pyspark/files.py b/python/pyspark/files.py
index 001b7a2..89bcbcf 100644
--- a/python/pyspark/files.py
+++ b/python/pyspark/files.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 import os
 
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/java_gateway.py
----------------------------------------------------------------------
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 2329e53..e503fb7 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 import os
 import sys
 from subprocess import Popen, PIPE

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/rdd.py
----------------------------------------------------------------------
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index a9fec17..c6a6b24 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 from base64 import standard_b64encode as b64enc
 import copy
 from collections import defaultdict

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/serializers.py
----------------------------------------------------------------------
diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py
index 5a95144..fecacd1 100644
--- a/python/pyspark/serializers.py
+++ b/python/pyspark/serializers.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 import struct
 import cPickle
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/shell.py
----------------------------------------------------------------------
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index 54ff1bf..cc8cd9e 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 """
 An interactive shell.
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/tests.py
----------------------------------------------------------------------
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index 1e34d47..dfd841b 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 """
 Unit tests for PySpark; additional tests are implemented as doctests in
 individual modules.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/pyspark/worker.py
----------------------------------------------------------------------
diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py
index 379bbfd..75d692b 100644
--- a/python/pyspark/worker.py
+++ b/python/pyspark/worker.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 """
 Worker that receives input from Piped RDD.
 """

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/python/run-tests
----------------------------------------------------------------------
diff --git a/python/run-tests b/python/run-tests
index a3a9ff5..1ee947d 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -1,6 +1,24 @@
 #!/usr/bin/env bash
 
-# Figure out where the Scala framework is installed
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+# Figure out where the Spark framework is installed
 FWDIR="$(cd `dirname $0`; cd ../; pwd)"
 
 FAILED=0

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/repl-bin/pom.xml
----------------------------------------------------------------------
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
index 7a72803..7c4e722 100644
--- a/repl-bin/pom.xml
+++ b/repl-bin/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
 <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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/repl-bin/src/deb/bin/run
----------------------------------------------------------------------
diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run
index c54c9e9..0e5100e 100755
--- a/repl-bin/src/deb/bin/run
+++ b/repl-bin/src/deb/bin/run
@@ -1,6 +1,23 @@
 #!/bin/bash
 
-SCALA_VERSION=2.9.2
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+SCALA_VERSION=2.9.3
 
 # Figure out where the Scala framework is installed
 FWDIR="$(cd `dirname $0`; pwd)"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/repl-bin/src/deb/bin/spark-executor
----------------------------------------------------------------------
diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor
index 47b9ccc..d3c1c23 100755
--- a/repl-bin/src/deb/bin/spark-executor
+++ b/repl-bin/src/deb/bin/spark-executor
@@ -1,5 +1,22 @@
 #!/bin/bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 FWDIR="$(cd `dirname $0`; pwd)"
 echo "Running spark-executor with framework dir = $FWDIR"
 exec $FWDIR/run spark.executor.MesosExecutorBackend

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/repl-bin/src/deb/bin/spark-shell
----------------------------------------------------------------------
diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell
index 219c66e..8b25854 100755
--- a/repl-bin/src/deb/bin/spark-shell
+++ b/repl-bin/src/deb/bin/spark-shell
@@ -1,4 +1,21 @@
 #!/bin/bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 FWDIR="$(cd `dirname $0`; pwd)"
 exec $FWDIR/run spark.repl.Main "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/repl/pom.xml
----------------------------------------------------------------------
diff --git a/repl/pom.xml b/repl/pom.xml
index 92a2020..7d8da03 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
 <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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala
----------------------------------------------------------------------
diff --git a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala
index 0e9aa86..274bc58 100644
--- a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala
+++ b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.repl
 
 import java.io.{ByteArrayOutputStream, InputStream}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/repl/src/main/scala/spark/repl/Main.scala
----------------------------------------------------------------------
diff --git a/repl/src/main/scala/spark/repl/Main.scala b/repl/src/main/scala/spark/repl/Main.scala
index 58809ab..d824d62 100644
--- a/repl/src/main/scala/spark/repl/Main.scala
+++ b/repl/src/main/scala/spark/repl/Main.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.repl
 
 import scala.collection.mutable.Set

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/repl/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties
index cfb1a39..a6d33e6 100644
--- a/repl/src/test/resources/log4j.properties
+++ b/repl/src/test/resources/log4j.properties
@@ -1,4 +1,21 @@
-# Set everything to be logged to the repl/target/unit-tests.log 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the repl/target/unit-tests.log
 log4j.rootCategory=INFO, file
 log4j.appender.file=org.apache.log4j.FileAppender
 log4j.appender.file.append=false

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/repl/src/test/scala/spark/repl/ReplSuite.scala
----------------------------------------------------------------------
diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala
index f46e6d8..80ae605 100644
--- a/repl/src/test/scala/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/spark/repl/ReplSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.repl
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/run
----------------------------------------------------------------------
diff --git a/run b/run
index 6b5bc01..72ee76d 100755
--- a/run
+++ b/run
@@ -1,5 +1,22 @@
 #!/bin/bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Figure out where the Scala framework is installed
 FWDIR="$(cd `dirname $0`; pwd)"
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/run.cmd
----------------------------------------------------------------------
diff --git a/run.cmd b/run.cmd
index cc5605f..c91764e 100644
--- a/run.cmd
+++ b/run.cmd
@@ -1,2 +1,20 @@
 @echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
 cmd /V /E /C %~dp0run2.cmd %*

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/run2.cmd
----------------------------------------------------------------------
diff --git a/run2.cmd b/run2.cmd
index a9c4df1..dc5e509 100644
--- a/run2.cmd
+++ b/run2.cmd
@@ -1,5 +1,22 @@
 @echo off
 
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
 set SCALA_VERSION=2.9.3
 
 rem Figure out where the Spark framework is installed

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/sbt/sbt
----------------------------------------------------------------------
diff --git a/sbt/sbt b/sbt/sbt
index 523fbb3..8c9a428 100755
--- a/sbt/sbt
+++ b/sbt/sbt
@@ -1,8 +1,28 @@
 #!/bin/bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 EXTRA_ARGS=""
 if [ "$MESOS_HOME" != "" ]; then
   EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java"
 fi
+
 export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd)
 export SPARK_TESTING=1  # To put test classes on classpath
+
 java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/sbt/sbt.cmd
----------------------------------------------------------------------
diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd
index ce3ae70..56a16d7 100644
--- a/sbt/sbt.cmd
+++ b/sbt/sbt.cmd
@@ -1,5 +1,25 @@
 @echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
 set EXTRA_ARGS=
 if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\java
+
 set SPARK_HOME=%~dp0..
+
 java -Xmx1200M -XX:MaxPermSize=200m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/spark-executor
----------------------------------------------------------------------
diff --git a/spark-executor b/spark-executor
index b66c374..feccbf5 100755
--- a/spark-executor
+++ b/spark-executor
@@ -1,4 +1,22 @@
 #!/bin/sh
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 FWDIR="`dirname $0`"
 echo "Running spark-executor with framework dir = $FWDIR"
 exec $FWDIR/run spark.executor.MesosExecutorBackend

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/spark-shell
----------------------------------------------------------------------
diff --git a/spark-shell b/spark-shell
index a8e7214..5371fc5 100755
--- a/spark-shell
+++ b/spark-shell
@@ -1,4 +1,22 @@
 #!/bin/bash --posix
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 #
 # Shell script for starting the Spark Shell REPL
 # Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}
@@ -6,6 +24,7 @@
 # Options:
 #    -c <cores>    Set the number of cores for REPL to use
 #
+
 FWDIR="`dirname $0`"
 
 for o in "$@"; do

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/spark-shell.cmd
----------------------------------------------------------------------
diff --git a/spark-shell.cmd b/spark-shell.cmd
index 34697d5..ec65eab 100644
--- a/spark-shell.cmd
+++ b/spark-shell.cmd
@@ -1,4 +1,22 @@
 @echo off
+
+rem
+rem Licensed to the Apache Software Foundation (ASF) under one or more
+rem contributor license agreements.  See the NOTICE file distributed with
+rem this work for additional information regarding copyright ownership.
+rem The ASF licenses this file to You under the Apache License, Version 2.0
+rem (the "License"); you may not use this file except in compliance with
+rem the License.  You may obtain a copy of the License at
+rem
+rem    http://www.apache.org/licenses/LICENSE-2.0
+rem
+rem Unless required by applicable law or agreed to in writing, software
+rem distributed under the License is distributed on an "AS IS" BASIS,
+rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+rem See the License for the specific language governing permissions and
+rem limitations under the License.
+rem
+
 set FWDIR=%~dp0
 set SPARK_LAUNCH_WITH_SCALA=1
 cmd /V /E /C %FWDIR%run2.cmd spark.repl.Main %*


[30/50] [abbrv] git commit: Added field to master UI with link to job UI

Posted by ma...@apache.org.
Added field to master UI with link to job UI


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/39557112
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/39557112
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/39557112

Branch: refs/heads/master
Commit: 39557112501901da7f9b4be6159a5a0be5511b42
Parents: e3d3e6f
Author: Karen Feng <ka...@gmail.com>
Authored: Mon Jul 15 15:47:21 2013 -0700
Committer: Karen Feng <ka...@gmail.com>
Committed: Mon Jul 15 15:47:21 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/39557112/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
index 7545ecf..434f600 100644
--- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
@@ -26,8 +26,8 @@ private[spark] class IndexPage(parent: MasterWebUI) {
     val workers = state.workers.sortBy(_.id)
     val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers)
 
-    val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User",
-      "State", "Duration")
+    val appHeaders = Seq("ID", "Job UI", "Description", "Cores", "Memory per Node", "Submit Time",
+      "User", "State", "Duration")
     val activeApps = state.activeApps.sortBy(_.startTime).reverse
     val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps)
     val completedApps = state.completedApps.sortBy(_.endTime).reverse
@@ -103,6 +103,9 @@ private[spark] class IndexPage(parent: MasterWebUI) {
       <td>
         <a href={"app?appId=" + app.id}>{app.id}</a>
       </td>
+      <td>
+        <a href={app.appUiUrl}>{app.appUiUrl}</a>
+      </td>
       <td>{app.desc.name}</td>
       <td>
         {app.coresGranted}


[29/50] [abbrv] git commit: Merge pull request #702 from karenfeng/ui-fixes

Posted by ma...@apache.org.
Merge pull request #702 from karenfeng/ui-fixes

Adds app name in HTML page titles on job web UI

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/e3d3e6f0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/e3d3e6f0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/e3d3e6f0

Branch: refs/heads/master
Commit: e3d3e6f0ab34f0fe083ef9feb31b9e9fd257519f
Parents: 4698a0d b2aaa11
Author: Matei Zaharia <ma...@gmail.com>
Authored: Mon Jul 15 14:59:44 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Mon Jul 15 14:59:44 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/ui/UIUtils.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[35/50] [abbrv] git commit: Throw a more meaningful message when runJob is called to launch tasks on non-existent partitions.

Posted by ma...@apache.org.
Throw a more meaningful message when runJob is called to launch tasks on non-existent partitions.


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/69316603
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/69316603
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/69316603

Branch: refs/heads/master
Commit: 69316603d6bf11ecf1ea3dab63df178bad835e2d
Parents: ed8415b
Author: Reynold Xin <re...@gmail.com>
Authored: Mon Jul 15 22:50:11 2013 -0700
Committer: Reynold Xin <re...@gmail.com>
Committed: Mon Jul 15 22:50:11 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/scheduler/DAGScheduler.scala | 9 +++++++++
 core/src/test/scala/spark/RDDSuite.scala               | 6 ++++++
 2 files changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/69316603/core/src/main/scala/spark/scheduler/DAGScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
index 3d3b9ea..8173ef7 100644
--- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
@@ -251,6 +251,15 @@ class DAGScheduler(
     if (partitions.size == 0) {
       return
     }
+
+    // Check to make sure we are not launching a task on a partition that does not exist.
+    val maxPartitions = finalRdd.partitions.length
+    partitions.find(p => p >= maxPartitions).foreach { p =>
+      throw new IllegalArgumentException(
+        "Attempting to access a non-existent partition: " + p + ". " +
+        "Total number of partitions: " + maxPartitions)
+    }
+
     val (toSubmit, waiter) = prepareJob(
         finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties)
     eventQueue.put(toSubmit)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/69316603/core/src/test/scala/spark/RDDSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala
index aa3ee5f..7f7d4c8 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/spark/RDDSuite.scala
@@ -302,4 +302,10 @@ class RDDSuite extends FunSuite with SharedSparkContext {
       assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements")
     }
   }
+
+  test("runJob on an invalid partition") {
+    intercept[IllegalArgumentException] {
+      sc.runJob(sc.parallelize(1 to 10, 2), {iter: Iterator[Int] => iter.size}, Seq(0, 1, 2), false)
+    }
+  }
 }


[44/50] [abbrv] Add Apache license headers and LICENSE and NOTICE files

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockException.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockException.scala b/core/src/main/scala/spark/storage/BlockException.scala
index f275d47..8ebfaf3 100644
--- a/core/src/main/scala/spark/storage/BlockException.scala
+++ b/core/src/main/scala/spark/storage/BlockException.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 private[spark]

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockFetchTracker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala
index 0718156..265e554 100644
--- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala
+++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 private[spark] trait BlockFetchTracker {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockFetcherIterator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala
index bec8762..1965c5b 100644
--- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala
+++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala
index 9b39d3a..e4ffa57 100644
--- a/core/src/main/scala/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/spark/storage/BlockManager.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.io.{InputStream, OutputStream}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockManagerId.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala
index 1e557d6..b36a617 100644
--- a/core/src/main/scala/spark/storage/BlockManagerId.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerId.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockManagerMaster.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
index 58888b1..3186f7c 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import akka.actor.ActorRef

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
index 0d4384b..244000d 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.util.{HashMap => JHashMap}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockManagerMessages.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala
index 0010726..01de4cc 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.io.{Externalizable, ObjectInput, ObjectOutput}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala
index b264d1d..45cffad 100644
--- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import akka.actor.Actor

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockManagerWorker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala
index 3057ade..39064bc 100644
--- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala
index 30d7500..ab72dbb 100644
--- a/core/src/main/scala/spark/storage/BlockMessage.scala
+++ b/core/src/main/scala/spark/storage/BlockMessage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockMessageArray.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala
index ee0c5ff..b0229d6 100644
--- a/core/src/main/scala/spark/storage/BlockMessageArray.scala
+++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockObjectWriter.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala
index 42e2b07..01ed6e8 100644
--- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala
+++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/BlockStore.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/BlockStore.scala b/core/src/main/scala/spark/storage/BlockStore.scala
index 8188d35..c8db002 100644
--- a/core/src/main/scala/spark/storage/BlockStore.scala
+++ b/core/src/main/scala/spark/storage/BlockStore.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/DiskStore.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala
index da859ee..3495d65 100644
--- a/core/src/main/scala/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/spark/storage/DiskStore.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/MemoryStore.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala
index eba5ee5..b5a86b8 100644
--- a/core/src/main/scala/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/spark/storage/MemoryStore.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.util.LinkedHashMap

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/PutResult.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/PutResult.scala b/core/src/main/scala/spark/storage/PutResult.scala
index 76f2360..3a0974f 100644
--- a/core/src/main/scala/spark/storage/PutResult.scala
+++ b/core/src/main/scala/spark/storage/PutResult.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/ShuffleBlockManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala
index 44638e0..8a7a6f9 100644
--- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import spark.serializer.Serializer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/StorageLevel.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala
index cc0c354..f526509 100644
--- a/core/src/main/scala/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/spark/storage/StorageLevel.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/StorageUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala
index aca16f5..2aeed4e 100644
--- a/core/src/main/scala/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/spark/storage/StorageUtils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import spark.{Utils, SparkContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/storage/ThreadingTest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala
index 5c406e6..b3ab1ff 100644
--- a/core/src/main/scala/spark/storage/ThreadingTest.scala
+++ b/core/src/main/scala/spark/storage/ThreadingTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import akka.actor._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/JettyUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala
index bc6f9c1..ca6088a 100644
--- a/core/src/main/scala/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/spark/ui/JettyUtils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui
 
 import annotation.tailrec

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/Page.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala
index ed8f918..a31e750 100644
--- a/core/src/main/scala/spark/ui/Page.scala
+++ b/core/src/main/scala/spark/ui/Page.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui
 
-private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value }
\ No newline at end of file
+private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/SparkUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala
index 874e5ba..9396f22 100644
--- a/core/src/main/scala/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/spark/ui/SparkUI.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui
 
 import javax.servlet.http.HttpServletRequest

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/UIUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala
index fa46e24..b1d1195 100644
--- a/core/src/main/scala/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/spark/ui/UIUtils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui
 
 import scala.xml.Node
@@ -115,4 +132,4 @@ private[spark] object UIUtils {
       </tbody>
     </table>
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala
index 8bbc6ce..a80e2d7 100644
--- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui
 
 import scala.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
index 6b8b9f0..5ae7935 100644
--- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
+++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui.env
 
 import javax.servlet.http.HttpServletRequest

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/jobs/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala
index 2df5f01..f31af3c 100644
--- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala
+++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui.jobs
 
 import java.util.Date

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala
index 84730cc..44dcf82 100644
--- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala
+++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui.jobs
 
 import akka.util.Duration
@@ -124,4 +141,4 @@ private[spark] class JobProgressListener extends SparkListener {
     }
     return false // No tasks have finished for this stage
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/jobs/StagePage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala
index 51b82b6..292966f 100644
--- a/core/src/main/scala/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui.jobs
 
 import java.util.Date

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala
index 3d67888..49ed069 100644
--- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala
+++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui.storage
 
 import akka.util.Duration

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/storage/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala
index d284134..4e0360d 100644
--- a/core/src/main/scala/spark/ui/storage/IndexPage.scala
+++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui.storage
 
 import javax.servlet.http.HttpServletRequest

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ui/storage/RDDPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala
index 428db6f..003be54 100644
--- a/core/src/main/scala/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui.storage
 
 import javax.servlet.http.HttpServletRequest

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/AkkaUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala
index 18fc967..9233277 100644
--- a/core/src/main/scala/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/spark/util/AkkaUtils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import akka.actor.{ActorSystem, ExtendedActorSystem}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/BoundedPriorityQueue.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala
index 4bc5db8..0575497 100644
--- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala
+++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import java.io.Serializable

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/ByteBufferInputStream.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/spark/util/ByteBufferInputStream.scala
index d7e6749..47a28e2 100644
--- a/core/src/main/scala/spark/util/ByteBufferInputStream.scala
+++ b/core/src/main/scala/spark/util/ByteBufferInputStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import java.io.InputStream

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/CompletionIterator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/CompletionIterator.scala b/core/src/main/scala/spark/util/CompletionIterator.scala
index 8139183..2104508 100644
--- a/core/src/main/scala/spark/util/CompletionIterator.scala
+++ b/core/src/main/scala/spark/util/CompletionIterator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 /**
@@ -22,4 +39,4 @@ object CompletionIterator {
       def completion() = completionFunction
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/Distribution.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala
index 24738b4..5d4d7a6 100644
--- a/core/src/main/scala/spark/util/Distribution.scala
+++ b/core/src/main/scala/spark/util/Distribution.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import java.io.PrintStream
@@ -62,4 +79,4 @@ object Distribution {
     quantiles.foreach{q => out.print(q + "\t")}
     out.println
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/IdGenerator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/IdGenerator.scala b/core/src/main/scala/spark/util/IdGenerator.scala
index b6e309f..3422280 100644
--- a/core/src/main/scala/spark/util/IdGenerator.scala
+++ b/core/src/main/scala/spark/util/IdGenerator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import java.util.concurrent.atomic.AtomicInteger

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/IntParam.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/IntParam.scala b/core/src/main/scala/spark/util/IntParam.scala
index 0427646..daf0d58 100644
--- a/core/src/main/scala/spark/util/IntParam.scala
+++ b/core/src/main/scala/spark/util/IntParam.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/MemoryParam.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/MemoryParam.scala b/core/src/main/scala/spark/util/MemoryParam.scala
index 3726738..2985623 100644
--- a/core/src/main/scala/spark/util/MemoryParam.scala
+++ b/core/src/main/scala/spark/util/MemoryParam.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import spark.Utils

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/MetadataCleaner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala
index dafa906..92909e0 100644
--- a/core/src/main/scala/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/spark/util/MetadataCleaner.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/NextIterator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/spark/util/NextIterator.scala
index 48b5018..22163ec 100644
--- a/core/src/main/scala/spark/util/NextIterator.scala
+++ b/core/src/main/scala/spark/util/NextIterator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 /** Provides a basic/boilerplate Iterator implementation. */
@@ -68,4 +85,4 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] {
     gotNext = false
     nextValue
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/RateLimitedOutputStream.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala
index e3f00ea..00f782b 100644
--- a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala
+++ b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import scala.annotation.tailrec

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/SerializableBuffer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/SerializableBuffer.scala b/core/src/main/scala/spark/util/SerializableBuffer.scala
index 09d588f..7e68426 100644
--- a/core/src/main/scala/spark/util/SerializableBuffer.scala
+++ b/core/src/main/scala/spark/util/SerializableBuffer.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/StatCounter.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/spark/util/StatCounter.scala
index 2b98034..76358d4 100644
--- a/core/src/main/scala/spark/util/StatCounter.scala
+++ b/core/src/main/scala/spark/util/StatCounter.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/TimeStampedHashMap.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala
index 92dfaa6..cc79091 100644
--- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala
+++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import java.util.concurrent.ConcurrentHashMap

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/TimeStampedHashSet.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/spark/util/TimeStampedHashSet.scala
index 5f1cc93..41e3fd8 100644
--- a/core/src/main/scala/spark/util/TimeStampedHashSet.scala
+++ b/core/src/main/scala/spark/util/TimeStampedHashSet.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import scala.collection.mutable.Set

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/util/Vector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/spark/util/Vector.scala
index 835822e..ed49386 100644
--- a/core/src/main/scala/spark/util/Vector.scala
+++ b/core/src/main/scala/spark/util/Vector.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 class Vector(val elements: Array[Double]) extends Serializable {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/resources/fairscheduler.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/fairscheduler.xml b/core/src/test/resources/fairscheduler.xml
index 6e573b1..996ffb1 100644
--- a/core/src/test/resources/fairscheduler.xml
+++ b/core/src/test/resources/fairscheduler.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
 <allocations>
 <pool name="1">
     <minShare>2</minShare>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties
index d05cf3d..f6fef03 100644
--- a/core/src/test/resources/log4j.properties
+++ b/core/src/test/resources/log4j.properties
@@ -1,4 +1,21 @@
-# Set everything to be logged to the file core/target/unit-tests.log 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file core/target/unit-tests.log
 log4j.rootCategory=INFO, file
 log4j.appender.file=org.apache.log4j.FileAppender
 log4j.appender.file.append=false

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/AccumulatorSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala
index ac8ae7d..0af175f 100644
--- a/core/src/test/scala/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/spark/AccumulatorSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/BroadcastSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/BroadcastSuite.scala b/core/src/test/scala/spark/BroadcastSuite.scala
index 362a31f..785721e 100644
--- a/core/src/test/scala/spark/BroadcastSuite.scala
+++ b/core/src/test/scala/spark/BroadcastSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/CheckpointSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala
index 28a7b21..a84c89e 100644
--- a/core/src/test/scala/spark/CheckpointSuite.scala
+++ b/core/src/test/scala/spark/CheckpointSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/ClosureCleanerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/ClosureCleanerSuite.scala b/core/src/test/scala/spark/ClosureCleanerSuite.scala
index b2d0dd4..7d2831e 100644
--- a/core/src/test/scala/spark/ClosureCleanerSuite.scala
+++ b/core/src/test/scala/spark/ClosureCleanerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io.NotSerializableException

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/DistributedSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala
index 0024ede..e11efe4 100644
--- a/core/src/test/scala/spark/DistributedSuite.scala
+++ b/core/src/test/scala/spark/DistributedSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import network.ConnectionManagerId

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/DriverSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala
index 31c3dd7..ed16b9d 100644
--- a/core/src/test/scala/spark/DriverSuite.scala
+++ b/core/src/test/scala/spark/DriverSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io.File

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/FailureSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala
index 8c1445a..6c847b8 100644
--- a/core/src/test/scala/spark/FailureSuite.scala
+++ b/core/src/test/scala/spark/FailureSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/FileServerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala
index c7855a7..242ae97 100644
--- a/core/src/test/scala/spark/FileServerSuite.scala
+++ b/core/src/test/scala/spark/FileServerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import com.google.common.io.Files

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/FileSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala
index e61ff77..1e2c257 100644
--- a/core/src/test/scala/spark/FileSuite.scala
+++ b/core/src/test/scala/spark/FileSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io.{FileWriter, PrintWriter, File}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/JavaAPISuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java
index d306124..aaf03e6 100644
--- a/core/src/test/scala/spark/JavaAPISuite.java
+++ b/core/src/test/scala/spark/JavaAPISuite.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/KryoSerializerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala
index 327e2ff..c3323dc 100644
--- a/core/src/test/scala/spark/KryoSerializerSuite.scala
+++ b/core/src/test/scala/spark/KryoSerializerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import scala.collection.mutable

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/LocalSparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala
index bd18422..ddc212d 100644
--- a/core/src/test/scala/spark/LocalSparkContext.scala
+++ b/core/src/test/scala/spark/LocalSparkContext.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.Suite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/MapOutputTrackerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala
index 6e585e1..ce6cec0 100644
--- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/PairRDDFunctionsSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala
index 682d274..b102eaf 100644
--- a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala
+++ b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import scala.collection.mutable.ArrayBuffer


[47/50] [abbrv] Add Apache license headers and LICENSE and NOTICE files

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala
index f1c86de..df55be1 100644
--- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapred.JobConf

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/java/spark/network/netty/FileClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java
index a4bb4bc..0625a6d 100644
--- a/core/src/main/java/spark/network/netty/FileClient.java
+++ b/core/src/main/java/spark/network/netty/FileClient.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty;
 
 import io.netty.bootstrap.Bootstrap;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java
index af25baf..05ad4b6 100644
--- a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java
+++ b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty;
 
 import io.netty.buffer.BufType;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/java/spark/network/netty/FileClientHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java
index 9fc9449..e8cd980 100644
--- a/core/src/main/java/spark/network/netty/FileClientHandler.java
+++ b/core/src/main/java/spark/network/netty/FileClientHandler.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty;
 
 import io.netty.buffer.ByteBuf;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/java/spark/network/netty/FileServer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java
index dd3a557..9f009a6 100644
--- a/core/src/main/java/spark/network/netty/FileServer.java
+++ b/core/src/main/java/spark/network/netty/FileServer.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty;
 
 import java.net.InetSocketAddress;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java
index 8f1f5c6..50c57a8 100644
--- a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java
+++ b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty;
 
 import io.netty.channel.ChannelInitializer;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/java/spark/network/netty/FileServerHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/spark/network/netty/FileServerHandler.java
index a78eddb..176ba8d 100644
--- a/core/src/main/java/spark/network/netty/FileServerHandler.java
+++ b/core/src/main/java/spark/network/netty/FileServerHandler.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/java/spark/network/netty/PathResolver.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/spark/network/netty/PathResolver.java
index 3024116..f446c55 100755
--- a/core/src/main/java/spark/network/netty/PathResolver.java
+++ b/core/src/main/java/spark/network/netty/PathResolver.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/Accumulators.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala
index 57c6df3..6ff92ce 100644
--- a/core/src/main/scala/spark/Accumulators.scala
+++ b/core/src/main/scala/spark/Accumulators.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/Aggregator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/spark/Aggregator.scala
index df8ce9c..136b4da 100644
--- a/core/src/main/scala/spark/Aggregator.scala
+++ b/core/src/main/scala/spark/Aggregator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.util.{HashMap => JHashMap}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
index 3239f4c..8f6953b 100644
--- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
+++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import scala.collection.mutable.ArrayBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/Cache.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala
index 20d677a..b0c83ce 100644
--- a/core/src/main/scala/spark/Cache.scala
+++ b/core/src/main/scala/spark/Cache.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.util.concurrent.atomic.AtomicInteger

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/CacheManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala
index f7a2b7e..8131480 100644
--- a/core/src/main/scala/spark/CacheManager.scala
+++ b/core/src/main/scala/spark/CacheManager.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import scala.collection.mutable.{ArrayBuffer, HashSet}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ClosureCleaner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala
index d5e7132..8b39241 100644
--- a/core/src/main/scala/spark/ClosureCleaner.scala
+++ b/core/src/main/scala/spark/ClosureCleaner.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.lang.reflect.Field

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/Dependency.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala
index 2af44aa..d17e70a 100644
--- a/core/src/main/scala/spark/Dependency.scala
+++ b/core/src/main/scala/spark/Dependency.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/DoubleRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala
index 178d31a..93ef097 100644
--- a/core/src/main/scala/spark/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import spark.partial.BoundedDouble

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/FetchFailedException.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/spark/FetchFailedException.scala
index 40b0193..a2dae6c 100644
--- a/core/src/main/scala/spark/FetchFailedException.scala
+++ b/core/src/main/scala/spark/FetchFailedException.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import spark.storage.BlockManagerId

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/HadoopWriter.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala
index 5e8396e..b1fe007 100644
--- a/core/src/main/scala/spark/HadoopWriter.scala
+++ b/core/src/main/scala/spark/HadoopWriter.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.mapred
 
 import org.apache.hadoop.fs.FileSystem

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/HttpFileServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/spark/HttpFileServer.scala
index 00901d9..a13a7a2 100644
--- a/core/src/main/scala/spark/HttpFileServer.scala
+++ b/core/src/main/scala/spark/HttpFileServer.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io.{File}
@@ -42,4 +59,4 @@ private[spark] class HttpFileServer extends Logging {
     return dir + "/" + file.getName
   }
   
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/HttpServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/HttpServer.scala b/core/src/main/scala/spark/HttpServer.scala
index 4e0507c..c9dffbc 100644
--- a/core/src/main/scala/spark/HttpServer.scala
+++ b/core/src/main/scala/spark/HttpServer.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io.File

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/JavaSerializer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/JavaSerializer.scala b/core/src/main/scala/spark/JavaSerializer.scala
index b04a27d..04c5f44 100644
--- a/core/src/main/scala/spark/JavaSerializer.scala
+++ b/core/src/main/scala/spark/JavaSerializer.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/KryoSerializer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala
index c7dbcc6..ee37da7 100644
--- a/core/src/main/scala/spark/KryoSerializer.scala
+++ b/core/src/main/scala/spark/KryoSerializer.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/Logging.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala
index 0fc8c31..79b0362 100644
--- a/core/src/main/scala/spark/Logging.scala
+++ b/core/src/main/scala/spark/Logging.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.slf4j.Logger

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/MapOutputTracker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala
index fde597f..2c417e3 100644
--- a/core/src/main/scala/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/spark/MapOutputTracker.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/PairRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index 8b313c6..6b0cc2f 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/spark/PairRDDFunctions.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/Partition.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/Partition.scala b/core/src/main/scala/spark/Partition.scala
index e384308..2a4edce 100644
--- a/core/src/main/scala/spark/Partition.scala
+++ b/core/src/main/scala/spark/Partition.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/Partitioner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala
index 6f8cd17..660af70 100644
--- a/core/src/main/scala/spark/Partitioner.scala
+++ b/core/src/main/scala/spark/Partitioner.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/RDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala
index 8aa7726..ca7cdd6 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/spark/RDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/RDDCheckpointData.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala
index 57e0405..b615f82 100644
--- a/core/src/main/scala/spark/RDDCheckpointData.scala
+++ b/core/src/main/scala/spark/RDDCheckpointData.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.apache.hadoop.fs.Path

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
index 2911f90..9f30b7f 100644
--- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io.EOFException

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/SerializableWritable.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/SerializableWritable.scala b/core/src/main/scala/spark/SerializableWritable.scala
index 8306fbf..0236611 100644
--- a/core/src/main/scala/spark/SerializableWritable.scala
+++ b/core/src/main/scala/spark/SerializableWritable.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/ShuffleFetcher.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala
index 9513a00..dcced03 100644
--- a/core/src/main/scala/spark/ShuffleFetcher.scala
+++ b/core/src/main/scala/spark/ShuffleFetcher.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import spark.executor.TaskMetrics

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/SizeEstimator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala
index f8a4c4e..6cc5756 100644
--- a/core/src/main/scala/spark/SizeEstimator.scala
+++ b/core/src/main/scala/spark/SizeEstimator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.lang.reflect.Field

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/SoftReferenceCache.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala
index 3dd0a4b..f41a379 100644
--- a/core/src/main/scala/spark/SoftReferenceCache.scala
+++ b/core/src/main/scala/spark/SoftReferenceCache.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import com.google.common.collect.MapMaker

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index 43e6af2..46b9935 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/SparkEnv.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala
index ec59b4f..f2bdc11 100644
--- a/core/src/main/scala/spark/SparkEnv.scala
+++ b/core/src/main/scala/spark/SparkEnv.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import collection.mutable

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/SparkException.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/SparkException.scala b/core/src/main/scala/spark/SparkException.scala
index aa7a16d..b7045ee 100644
--- a/core/src/main/scala/spark/SparkException.scala
+++ b/core/src/main/scala/spark/SparkException.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 class SparkException(message: String, cause: Throwable)

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/SparkFiles.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/SparkFiles.java b/core/src/main/scala/spark/SparkFiles.java
index 566aec6..f9b3f79 100644
--- a/core/src/main/scala/spark/SparkFiles.java
+++ b/core/src/main/scala/spark/SparkFiles.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/TaskContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala
index dd06090..b79f4ca 100644
--- a/core/src/main/scala/spark/TaskContext.scala
+++ b/core/src/main/scala/spark/TaskContext.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import executor.TaskMetrics

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/TaskEndReason.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala
index bb75ec2..3ad665d 100644
--- a/core/src/main/scala/spark/TaskEndReason.scala
+++ b/core/src/main/scala/spark/TaskEndReason.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import spark.executor.TaskMetrics

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/TaskState.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala
index 78eb33a..9df7d82 100644
--- a/core/src/main/scala/spark/TaskState.scala
+++ b/core/src/main/scala/spark/TaskState.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.apache.mesos.Protos.{TaskState => MesosTaskState}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala
index a36186b..e6a96a5 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/spark/Utils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
index 16692c0..392556f 100644
--- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/JavaPairRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
index 7605159..ccc511d 100644
--- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java
 
 import java.util.{List => JList}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/JavaRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala
index 626b499..c0bf2cf 100644
--- a/core/src/main/scala/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/JavaRDDLike.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
index 27f40ec..21b5abf 100644
--- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java
 
 import java.util.{List => JList, Comparator}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/JavaSparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
index 5f18b1e..fe182e7 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java
 
 import java.util.{Map => JMap}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java
index 97344e7..42b1de0 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java
+++ b/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java;
 
 import java.util.Arrays;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/StorageLevels.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/StorageLevels.java b/core/src/main/scala/spark/api/java/StorageLevels.java
index 5e5845a..f385636 100644
--- a/core/src/main/scala/spark/api/java/StorageLevels.java
+++ b/core/src/main/scala/spark/api/java/StorageLevels.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java;
 
 import spark.storage.StorageLevel;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java
index 3a8192b..8bc88d7 100644
--- a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java
+++ b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/DoubleFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFunction.java
index c6ef76d..1aa1e5d 100644
--- a/core/src/main/scala/spark/api/java/function/DoubleFunction.java
+++ b/core/src/main/scala/spark/api/java/function/DoubleFunction.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala
index e027cda..9eb0cfe 100644
--- a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala
+++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala
index 6044043..dda9871 100644
--- a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala
+++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/Function.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/Function.java b/core/src/main/scala/spark/api/java/function/Function.java
index dae8295..2a2ea0a 100644
--- a/core/src/main/scala/spark/api/java/function/Function.java
+++ b/core/src/main/scala/spark/api/java/function/Function.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function;
 
 import scala.reflect.ClassManifest;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/Function2.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/Function2.java b/core/src/main/scala/spark/api/java/function/Function2.java
index 69bf12c..952d31e 100644
--- a/core/src/main/scala/spark/api/java/function/Function2.java
+++ b/core/src/main/scala/spark/api/java/function/Function2.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function;
 
 import scala.reflect.ClassManifest;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java
index b3cc4df..4aad602 100644
--- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java
+++ b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function;
 
 import scala.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/PairFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/PairFunction.java b/core/src/main/scala/spark/api/java/function/PairFunction.java
index 9fc6df4..ccfe64e 100644
--- a/core/src/main/scala/spark/api/java/function/PairFunction.java
+++ b/core/src/main/scala/spark/api/java/function/PairFunction.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function;
 
 import scala.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/VoidFunction.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/VoidFunction.scala b/core/src/main/scala/spark/api/java/function/VoidFunction.scala
index b0096cf..f6fc0b0 100644
--- a/core/src/main/scala/spark/api/java/function/VoidFunction.scala
+++ b/core/src/main/scala/spark/api/java/function/VoidFunction.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function
 
 /**
@@ -13,4 +30,4 @@ abstract class VoidFunction[T] extends Serializable {
 // return Unit), so it is implicitly converted to a Function1[T, Unit]:
 object VoidFunction {
   implicit def toFunction[T](f: VoidFunction[T]) : Function1[T, Unit] = ((x : T) => f.call(x))
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala
index 923f5cd..1758a38 100644
--- a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala
+++ b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function
 
 import scala.runtime.AbstractFunction1

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala
index 2c6e9b1..b093567 100644
--- a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala
+++ b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.java.function
 
 import scala.runtime.AbstractFunction2

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/python/PythonPartitioner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala
index d618c09..31a719f 100644
--- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala
+++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.python
 
 import spark.Partitioner

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/python/PythonRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala
index 31d8ea8..af10822 100644
--- a/core/src/main/scala/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/spark/api/python/PythonRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.python
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala
index 85d1dfe..078ad45 100644
--- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala
+++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.api.python
 
 import java.io.{DataInputStream, IOException}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
index adcb2d2..6f7d385 100644
--- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.broadcast
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/broadcast/Broadcast.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala
index 415bde5..aba56a6 100644
--- a/core/src/main/scala/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/spark/broadcast/Broadcast.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.broadcast
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
index 5c6184c..d33d95c 100644
--- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
+++ b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.broadcast
 
 /**


[09/50] [abbrv] git commit: Increase PermGen size

Posted by ma...@apache.org.
Increase PermGen size


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/1889f9f4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/1889f9f4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/1889f9f4

Branch: refs/heads/master
Commit: 1889f9f450d3e28864a5ac7b5f9d3831cf7b17bf
Parents: 77c69ae
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Sat Jul 13 14:43:21 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Sat Jul 13 14:43:21 2013 -0700

----------------------------------------------------------------------
 sbt/sbt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/1889f9f4/sbt/sbt
----------------------------------------------------------------------
diff --git a/sbt/sbt b/sbt/sbt
index 850c58e..523fbb3 100755
--- a/sbt/sbt
+++ b/sbt/sbt
@@ -5,4 +5,4 @@ if [ "$MESOS_HOME" != "" ]; then
 fi
 export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd)
 export SPARK_TESTING=1  # To put test classes on classpath
-java -Xmx1200m -XX:MaxPermSize=250m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@"
+java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@"


[18/50] [abbrv] git commit: Fix a comment

Posted by ma...@apache.org.
Fix a comment


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/931e4c96
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/931e4c96
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/931e4c96

Branch: refs/heads/master
Commit: 931e4c96ef56302cc384086d1faa0f4ca3258e8b
Parents: c5c38d1
Author: Matei Zaharia <ma...@gmail.com>
Authored: Sun Jul 14 08:03:13 2013 +0000
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Sun Jul 14 08:03:13 2013 +0000

----------------------------------------------------------------------
 mllib/src/main/scala/spark/mllib/recommendation/ALS.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/931e4c96/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
index dec3701..21eb212 100644
--- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
@@ -161,7 +161,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
     for (productBlock <- 0 until numBlocks) {
       // Create an array of (product, Seq(Rating)) ratings
       val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray
-      // Sort them by user ID
+      // Sort them by product ID
       val ordering = new Ordering[(Int, ArrayBuffer[Rating])] {
         def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = a._1 - b._1
       }


[12/50] [abbrv] git commit: Merge remote-tracking branch 'origin/pr/662'

Posted by ma...@apache.org.
Merge remote-tracking branch 'origin/pr/662'

Conflicts:
	bin/compute-classpath.sh


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/cd28d9c1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/cd28d9c1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/cd28d9c1

Branch: refs/heads/master
Commit: cd28d9c14732a91416589bb3dc523533495240e8
Parents: 1889f9f 1107b4d
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Sat Jul 13 19:10:00 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Sat Jul 13 19:10:00 2013 -0700

----------------------------------------------------------------------
 bin/compute-classpath.sh | 111 +++++++++++++++++++++++-------------------
 bin/start-slave.sh       |   3 ++
 make-distribution.sh     |  39 +++++++++++++++
 project/SparkBuild.scala |   2 +-
 run                      |  44 ++++++-----------
 spark-shell              |  67 +++++++++++++++++++++++--
 6 files changed, 185 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cd28d9c1/bin/compute-classpath.sh
----------------------------------------------------------------------
diff --cc bin/compute-classpath.sh
index 75c58d1,47937a0..bd48b43
--- a/bin/compute-classpath.sh
+++ b/bin/compute-classpath.sh
@@@ -24,46 -23,70 +24,71 @@@ PYSPARK_DIR="$FWDIR/python
  
  # Build up classpath
  CLASSPATH="$SPARK_CLASSPATH"
- CLASSPATH="$CLASSPATH:$FWDIR/conf"
- CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes"
- if [ -n "$SPARK_TESTING" ] ; then
-   CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
-   CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes"
- fi
- CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources"
- CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes"
- CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
- CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
- CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar
- if [ -e "$FWDIR/lib_managed" ]; then
-   CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*"
-   CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*"
- fi
- CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*"
- # Add the shaded JAR for Maven builds
- if [ -e $REPL_BIN_DIR/target ]; then
-   for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
+ 
+ function dev_classpath {
+   CLASSPATH="$CLASSPATH:$FWDIR/conf"
+   CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes"
+   if [ -n "$SPARK_TESTING" ] ; then
+     CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
+     CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes"
+   fi
+   CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources"
+   CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes"
+   CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
+   CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
+   CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar
+   if [ -e "$FWDIR/lib_managed" ]; then
+     CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*"
+     CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*"
+   fi
+   CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*"
+   # Add the shaded JAR for Maven builds
+   if [ -e $REPL_BIN_DIR/target ]; then
+     for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
+       CLASSPATH="$CLASSPATH:$jar"
+     done
+     # The shaded JAR doesn't contain examples, so include those separately
+     EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
+     CLASSPATH+=":$EXAMPLES_JAR"
+   fi
+   CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
++  CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes"
+   for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
      CLASSPATH="$CLASSPATH:$jar"
    done
-   # The shaded JAR doesn't contain examples, so include those separately
-   EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
-   CLASSPATH+=":$EXAMPLES_JAR"
- fi
- CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
- CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes"
- for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
-   CLASSPATH="$CLASSPATH:$jar"
- done
  
- # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
- # to avoid the -sources and -doc packages that are built by publish-local.
- if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then
-   # Use the JAR from the SBT build
-   export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar`
- fi
- if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then
-   # Use the JAR from the Maven build
-   export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
+   # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
+   # to avoid the -sources and -doc packages that are built by publish-local.
+   if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then
+     # Use the JAR from the SBT build
+     export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar`
+   fi
+   if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then
+     # Use the JAR from the Maven build
+     export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
+   fi
+ 
+   # Add Scala standard library
+   if [ -z "$SCALA_LIBRARY_PATH" ]; then
+     if [ -z "$SCALA_HOME" ]; then
+       echo "SCALA_HOME is not set" >&2
+       exit 1
+     fi
+     SCALA_LIBRARY_PATH="$SCALA_HOME/lib"
+   fi
+   CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar"
+   CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar"
+   CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar"
+ }
+ 
+ function release_classpath {
+   CLASSPATH="$CLASSPATH:$FWDIR/jars/*"
+ }
+ 
+ if [ -f "$FWDIR/RELEASE" ]; then
+   release_classpath
+ else
+   dev_classpath
  fi
  
  # Add hadoop conf dir - else FileSystem.*, etc fail !

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cd28d9c1/make-distribution.sh
----------------------------------------------------------------------
diff --cc make-distribution.sh
index 0000000,8554758..feb13d5
mode 000000,100755..100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@@ -1,0 -1,38 +1,39 @@@
+ #!/bin/bash
+ #
+ # Script to create a binary distribution for easy deploys of Spark.
+ # The distribution directory defaults to dist/ but can be overridden below.
+ # The distribution contains fat (assembly) jars that include the Scala library,
+ # so it is completely self contained.
+ # It does not contain source or *.class files.
+ #
+ # Recommended deploy/testing procedure (standalone mode):
+ # 1) Rsync / deploy the dist/ dir to one host
+ # 2) cd to deploy dir; ./bin/start-master.sh
+ # 3) Verify master is up by visiting web page, ie http://master-ip:8080.  Note the spark:// URL.
+ # 4) ./bin/start-slave.sh 1 <<spark:// URL>>
+ # 5) MASTER="spark://my-master-ip:7077" ./spark-shell
+ 
+ # Figure out where the Spark framework is installed
+ FWDIR="$(cd `dirname $0`; pwd)"
+ DISTDIR="$FWDIR/dist"
+ 
+ # Get version from SBT
++export TERM=dumb   # Prevents color codes in SBT output
+ VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2)
+ echo "Making distribution for Spark $VERSION in $DISTDIR..."
+ 
+ # Build fat JAR
+ $FWDIR/sbt/sbt "repl/assembly"
+ 
+ # Make directories
+ rm -rf "$DISTDIR"
+ mkdir -p "$DISTDIR/jars"
+ echo "$VERSION" >$DISTDIR/RELEASE
+ 
+ # Copy jars
+ cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/"
+ 
+ # Copy other things
+ cp -r "$FWDIR/bin" "$DISTDIR"
+ cp -r "$FWDIR/conf" "$DISTDIR"
 -cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR"
++cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/cd28d9c1/project/SparkBuild.scala
----------------------------------------------------------------------
diff --cc project/SparkBuild.scala
index 641d379,6058fb1..bc9214d
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@@ -197,7 -198,7 +197,7 @@@ object SparkBuild extends Build 
    def replSettings = sharedSettings ++ Seq(
      name := "spark-repl",
      libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _)
-   )
 -  ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings
++  ) ++ assemblySettings ++ extraAssemblySettings
  
    def examplesSettings = sharedSettings ++ Seq(
      name := "spark-examples",


[37/50] [abbrv] git commit: Fix deprecation warning and style issues

Posted by ma...@apache.org.
Fix deprecation warning and style issues


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/f347cc3f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/f347cc3f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/f347cc3f

Branch: refs/heads/master
Commit: f347cc3f659d4414a21de26feadcbe23a130e622
Parents: 10c0593
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Tue Jul 16 10:53:30 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Tue Jul 16 10:53:30 2013 -0700

----------------------------------------------------------------------
 core/src/test/scala/spark/FileServerSuite.scala | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/f347cc3f/core/src/test/scala/spark/FileServerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala
index 9c24ca4..c7855a7 100644
--- a/core/src/test/scala/spark/FileServerSuite.scala
+++ b/core/src/test/scala/spark/FileServerSuite.scala
@@ -35,26 +35,26 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
     val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
     val result = sc.parallelize(testData).reduceByKey {
       val path = SparkFiles.get("FileServerSuite.txt")
-      val in  = new BufferedReader(new FileReader(path))
+      val in = new BufferedReader(new FileReader(path))
       val fileVal = in.readLine().toInt
       in.close()
       _ * fileVal + _ * fileVal
-    }.collect
+    }.collect()
     assert(result.toSet === Set((1,200), (2,300), (3,500)))
   }
 
   test("Distributing files locally using URL as input") {
     // addFile("file:///....")
     sc = new SparkContext("local[4]", "test")
-    sc.addFile((new File(tmpFile.toString)).toURL.toString)
+    sc.addFile(new File(tmpFile.toString).toURI.toString)
     val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
     val result = sc.parallelize(testData).reduceByKey {
       val path = SparkFiles.get("FileServerSuite.txt")
-      val in  = new BufferedReader(new FileReader(path))
+      val in = new BufferedReader(new FileReader(path))
       val fileVal = in.readLine().toInt
       in.close()
       _ * fileVal + _ * fileVal
-    }.collect
+    }.collect()
     assert(result.toSet === Set((1,200), (2,300), (3,500)))
   }
 
@@ -80,11 +80,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
     val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
     val result = sc.parallelize(testData).reduceByKey {
       val path = SparkFiles.get("FileServerSuite.txt")
-      val in  = new BufferedReader(new FileReader(path))
+      val in = new BufferedReader(new FileReader(path))
       val fileVal = in.readLine().toInt
       in.close()
       _ * fileVal + _ * fileVal
-    }.collect
+    }.collect()
     assert(result.toSet === Set((1,200), (2,300), (3,500)))
   }
 


[43/50] [abbrv] Add Apache license headers and LICENSE and NOTICE files

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/PartitioningSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala
index 99e433e..b1e0b2b 100644
--- a/core/src/test/scala/spark/PartitioningSuite.scala
+++ b/core/src/test/scala/spark/PartitioningSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/PipedRDDSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala
index d263bb0..35c0471 100644
--- a/core/src/test/scala/spark/PipedRDDSuite.scala
+++ b/core/src/test/scala/spark/PipedRDDSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/RDDSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala
index 7f7d4c8..cbddf4e 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/spark/RDDSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import scala.collection.mutable.HashMap

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/SharedSparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/SharedSparkContext.scala b/core/src/test/scala/spark/SharedSparkContext.scala
index 1da79f9..70c2451 100644
--- a/core/src/test/scala/spark/SharedSparkContext.scala
+++ b/core/src/test/scala/spark/SharedSparkContext.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.Suite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/ShuffleNettySuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/ShuffleNettySuite.scala b/core/src/test/scala/spark/ShuffleNettySuite.scala
index bfaffa9..6bad6c1 100644
--- a/core/src/test/scala/spark/ShuffleNettySuite.scala
+++ b/core/src/test/scala/spark/ShuffleNettySuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.BeforeAndAfterAll

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/ShuffleSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala
index 950218f..3a56c26 100644
--- a/core/src/test/scala/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/spark/ShuffleSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import scala.collection.mutable.ArrayBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/SizeEstimatorSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala
index b5c8525..1ef812d 100644
--- a/core/src/test/scala/spark/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/SortingSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala
index f7bf207..b933c4a 100644
--- a/core/src/test/scala/spark/SortingSuite.scala
+++ b/core/src/test/scala/spark/SortingSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/ThreadingSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/ThreadingSuite.scala b/core/src/test/scala/spark/ThreadingSuite.scala
index ff315b6..f2acd0b 100644
--- a/core/src/test/scala/spark/ThreadingSuite.scala
+++ b/core/src/test/scala/spark/ThreadingSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import java.util.concurrent.Semaphore

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/UnpersistSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/UnpersistSuite.scala b/core/src/test/scala/spark/UnpersistSuite.scala
index 94776e7..93977d1 100644
--- a/core/src/test/scala/spark/UnpersistSuite.scala
+++ b/core/src/test/scala/spark/UnpersistSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/UtilsSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala
index 1e1260f..31c3b25 100644
--- a/core/src/test/scala/spark/UtilsSuite.scala
+++ b/core/src/test/scala/spark/UtilsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import com.google.common.base.Charsets

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/ZippedPartitionsSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala
index 96cb295..5e6d7b0 100644
--- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala
+++ b/core/src/test/scala/spark/ZippedPartitionsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import scala.collection.immutable.NumericRange

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala
index 6afb0fa..dc8ca94 100644
--- a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala
+++ b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark
 
 import org.scalatest.{ BeforeAndAfter, FunSuite }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
index d27a253..d1276d5 100644
--- a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
+++ b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import scala.collection.immutable.NumericRange

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala
index 8e1ad27..8f81d0b 100644
--- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
index 30e6fef..f802b66 100644
--- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import scala.collection.mutable.{Map, HashMap}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
index 2474d74..0f855c3 100644
--- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import java.util.Properties

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala
index 8bd813f..14bb587 100644
--- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala
index 48aa67c..392d67d 100644
--- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
index 647bcaf..95a6eee 100644
--- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/storage/BlockManagerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala
index b9d5f96..b719d65 100644
--- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.storage
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/ui/UISuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala
index fc0c160..56c1fed 100644
--- a/core/src/test/scala/spark/ui/UISuite.scala
+++ b/core/src/test/scala/spark/ui/UISuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.ui
 
 import scala.util.{Failure, Success, Try}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/util/DistributionSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/util/DistributionSuite.scala b/core/src/test/scala/spark/util/DistributionSuite.scala
index cc6249b..6578b55 100644
--- a/core/src/test/scala/spark/util/DistributionSuite.scala
+++ b/core/src/test/scala/spark/util/DistributionSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/util/NextIteratorSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/util/NextIteratorSuite.scala b/core/src/test/scala/spark/util/NextIteratorSuite.scala
index ed5b36d..fdbd43d 100644
--- a/core/src/test/scala/spark/util/NextIteratorSuite.scala
+++ b/core/src/test/scala/spark/util/NextIteratorSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
index 794063f..4c00442 100644
--- a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
+++ b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.util
 
 import org.scalatest.FunSuite

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/docs/_plugins/copy_api_dirs.rb
----------------------------------------------------------------------
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
index c10ae59..45ef4bb 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 require 'fileutils'
 include FileUtils
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/ec2/spark-ec2
----------------------------------------------------------------------
diff --git a/ec2/spark-ec2 b/ec2/spark-ec2
index 2714f19..454057a 100755
--- a/ec2/spark-ec2
+++ b/ec2/spark-ec2
@@ -1,5 +1,6 @@
 #!/bin/sh
 
+#
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file
 # distributed with this work for additional information
@@ -15,6 +16,7 @@
 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 # See the License for the specific language governing permissions and
 # limitations under the License.
+#
 
 cd "`dirname $0`"
 PYTHONPATH="./third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" python ./spark_ec2.py $@

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/ec2/spark_ec2.py
----------------------------------------------------------------------
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 7affe6f..2ec3c00 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -1,6 +1,7 @@
 #!/usr/bin/env python
 # -*- coding: utf-8 -*-
 
+#
 # Licensed to the Apache Software Foundation (ASF) under one
 # or more contributor license agreements.  See the NOTICE file
 # distributed with this work for additional information
@@ -16,6 +17,7 @@
 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 # See the License for the specific language governing permissions and
 # limitations under the License.
+#
 
 from __future__ import with_statement
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 1976765..7a8d08f 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
 <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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/java/spark/examples/JavaHdfsLR.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/spark/examples/JavaHdfsLR.java
index 8b0a9b6..9485e0c 100644
--- a/examples/src/main/java/spark/examples/JavaHdfsLR.java
+++ b/examples/src/main/java/spark/examples/JavaHdfsLR.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples;
 
 import spark.api.java.JavaRDD;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/java/spark/examples/JavaKMeans.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/spark/examples/JavaKMeans.java
index 626034e..2d34776 100644
--- a/examples/src/main/java/spark/examples/JavaKMeans.java
+++ b/examples/src/main/java/spark/examples/JavaKMeans.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples;
 
 import scala.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/java/spark/examples/JavaLogQuery.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/spark/examples/JavaLogQuery.java
index 6b22e71..d22684d 100644
--- a/examples/src/main/java/spark/examples/JavaLogQuery.java
+++ b/examples/src/main/java/spark/examples/JavaLogQuery.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/java/spark/examples/JavaSparkPi.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/spark/examples/JavaSparkPi.java
index a15a967..d5f42fb 100644
--- a/examples/src/main/java/spark/examples/JavaSparkPi.java
+++ b/examples/src/main/java/spark/examples/JavaSparkPi.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples;
 
 import spark.api.java.JavaRDD;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/java/spark/examples/JavaTC.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/spark/examples/JavaTC.java b/examples/src/main/java/spark/examples/JavaTC.java
index b319bda..559d7f9 100644
--- a/examples/src/main/java/spark/examples/JavaTC.java
+++ b/examples/src/main/java/spark/examples/JavaTC.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples;
 
 import scala.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/java/spark/examples/JavaWordCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/spark/examples/JavaWordCount.java b/examples/src/main/java/spark/examples/JavaWordCount.java
index 9d4c7a2..1af370c 100644
--- a/examples/src/main/java/spark/examples/JavaWordCount.java
+++ b/examples/src/main/java/spark/examples/JavaWordCount.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples;
 
 import scala.Tuple2;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
index e24c6dd..096a9ae 100644
--- a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples;
 
 import spark.api.java.function.Function;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
index 3e57580..c54d3f3 100644
--- a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
+++ b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
index 15b82c8..1f4a991 100644
--- a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
+++ b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/BroadcastTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/BroadcastTest.scala b/examples/src/main/scala/spark/examples/BroadcastTest.scala
index ba59be1..911490c 100644
--- a/examples/src/main/scala/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/spark/examples/BroadcastTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark.SparkContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/CassandraTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala
index 0fe1833..104bfd5 100644
--- a/examples/src/main/scala/spark/examples/CassandraTest.scala
+++ b/examples/src/main/scala/spark/examples/CassandraTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import org.apache.hadoop.mapreduce.Job

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
index 21a90f2..67ddaec 100644
--- a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
+++ b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark.SparkContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/GroupByTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/GroupByTest.scala b/examples/src/main/scala/spark/examples/GroupByTest.scala
index a660365..5cee413 100644
--- a/examples/src/main/scala/spark/examples/GroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/GroupByTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark.SparkContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/HBaseTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala
index 6e91015..4dd6c24 100644
--- a/examples/src/main/scala/spark/examples/HBaseTest.scala
+++ b/examples/src/main/scala/spark/examples/HBaseTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark._
@@ -32,4 +49,4 @@ object HBaseTest {
 
     System.exit(0)
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/HdfsTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/HdfsTest.scala b/examples/src/main/scala/spark/examples/HdfsTest.scala
index dd61c46..2325833 100644
--- a/examples/src/main/scala/spark/examples/HdfsTest.scala
+++ b/examples/src/main/scala/spark/examples/HdfsTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/LocalALS.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/LocalALS.scala b/examples/src/main/scala/spark/examples/LocalALS.scala
index 2de810e..7a449a9 100644
--- a/examples/src/main/scala/spark/examples/LocalALS.scala
+++ b/examples/src/main/scala/spark/examples/LocalALS.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import scala.math.sqrt

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/LocalFileLR.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/LocalFileLR.scala b/examples/src/main/scala/spark/examples/LocalFileLR.scala
index f958ef9..c1f8d32 100644
--- a/examples/src/main/scala/spark/examples/LocalFileLR.scala
+++ b/examples/src/main/scala/spark/examples/LocalFileLR.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/LocalKMeans.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/spark/examples/LocalKMeans.scala
index 4849f21..0a0bc6f 100644
--- a/examples/src/main/scala/spark/examples/LocalKMeans.scala
+++ b/examples/src/main/scala/spark/examples/LocalKMeans.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/LocalLR.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/LocalLR.scala b/examples/src/main/scala/spark/examples/LocalLR.scala
index cd73f55..ab99bf1 100644
--- a/examples/src/main/scala/spark/examples/LocalLR.scala
+++ b/examples/src/main/scala/spark/examples/LocalLR.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/LocalPi.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/LocalPi.scala b/examples/src/main/scala/spark/examples/LocalPi.scala
index 9457472..ccd6969 100644
--- a/examples/src/main/scala/spark/examples/LocalPi.scala
+++ b/examples/src/main/scala/spark/examples/LocalPi.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import scala.math.random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/LogQuery.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/LogQuery.scala b/examples/src/main/scala/spark/examples/LogQuery.scala
index 6497596..e815ece 100644
--- a/examples/src/main/scala/spark/examples/LogQuery.scala
+++ b/examples/src/main/scala/spark/examples/LogQuery.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark.SparkContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
index a0aaf60..d0b1cf0 100644
--- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark.SparkContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
index 461b84a..d197bba 100644
--- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark.SparkContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
index 435675f..4641b82 100644
--- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark.SparkContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/SparkALS.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/SparkALS.scala b/examples/src/main/scala/spark/examples/SparkALS.scala
index 8fb3b0f..ba0dfd8 100644
--- a/examples/src/main/scala/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/spark/examples/SparkALS.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import scala.math.sqrt

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
index 3d080a0..ef6e09a 100644
--- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/SparkKMeans.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/SparkKMeans.scala b/examples/src/main/scala/spark/examples/SparkKMeans.scala
index 4161c59..38ed3b1 100644
--- a/examples/src/main/scala/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/spark/examples/SparkKMeans.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/SparkLR.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/SparkLR.scala b/examples/src/main/scala/spark/examples/SparkLR.scala
index 2f41aeb..52a0d69 100644
--- a/examples/src/main/scala/spark/examples/SparkLR.scala
+++ b/examples/src/main/scala/spark/examples/SparkLR.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/SparkPi.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala
index f598d2f..00560ac 100644
--- a/examples/src/main/scala/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/spark/examples/SparkPi.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import scala.math.random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/examples/SparkTC.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/examples/SparkTC.scala b/examples/src/main/scala/spark/examples/SparkTC.scala
index 911ae8f..bf988a9 100644
--- a/examples/src/main/scala/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/spark/examples/SparkTC.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.examples
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala
index 3b847fe..f97174a 100644
--- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import scala.collection.mutable.LinkedList

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
index 39c76fd..3ab4fc2 100644
--- a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import spark.util.IntParam

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
index 9389f8a..f5baec2 100644
--- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import spark.streaming.{Seconds, StreamingContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
index 9202e65..4929703 100644
--- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import java.util.Properties

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
index 704540c..150fb5e 100644
--- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import spark.streaming.{Seconds, StreamingContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
index f450e21..da36c8c 100644
--- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
+++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import spark.RDD

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
index 175281e..7fb680b 100644
--- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
+++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.streaming.examples
 
 import spark.util.IntParam


[17/50] [abbrv] git commit: Some optimizations to loading phase of ALS

Posted by ma...@apache.org.
Some optimizations to loading phase of ALS


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/c5c38d19
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/c5c38d19
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/c5c38d19

Branch: refs/heads/master
Commit: c5c38d1987137a1dc5eb66dd1065735a542ef9b5
Parents: b91a218
Author: Matei Zaharia <ma...@gmail.com>
Authored: Sun Jul 14 07:59:50 2013 +0000
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Sun Jul 14 07:59:50 2013 +0000

----------------------------------------------------------------------
 .../scala/spark/mllib/recommendation/ALS.scala  | 45 ++++++++++++++------
 1 file changed, 32 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c5c38d19/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
index 6c9fb23..dec3701 100644
--- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
@@ -2,6 +2,7 @@ package 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
@@ -34,6 +35,12 @@ private[recommendation] case class InLinkBlock(
 
 
 /**
+ * A more compact class to represent a rating than Tuple3[Int, Int, Double].
+ */
+private[recommendation] case class Rating(user: Int, product: Int, rating: Double)
+
+
+/**
  * Alternating Least Squares matrix factorization.
  *
  * This is a blocked implementation of the ALS factorization algorithm that groups the two sets
@@ -126,13 +133,13 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
    * Make the out-links table for a block of the users (or products) dataset given the list of
    * (user, product, rating) values for the users in that block (or the opposite for products).
    */
-  private def makeOutLinkBlock(numBlocks: Int, ratings: Array[(Int, Int, Double)]): OutLinkBlock = {
-    val userIds = ratings.map(_._1).distinct.sorted
+  private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating]): OutLinkBlock = {
+    val userIds = ratings.map(_.user).distinct.sorted
     val numUsers = userIds.length
     val userIdToPos = userIds.zipWithIndex.toMap
     val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks))
-    for ((u, p, r) <- ratings) {
-      shouldSend(userIdToPos(u))(p % numBlocks) = true
+    for (r <- ratings) {
+      shouldSend(userIdToPos(r.user))(r.product % numBlocks) = true
     }
     OutLinkBlock(userIds, shouldSend)
   }
@@ -141,18 +148,28 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
    * Make the in-links table for a block of the users (or products) dataset given a list of
    * (user, product, rating) values for the users in that block (or the opposite for products).
    */
-  private def makeInLinkBlock(numBlocks: Int, ratings: Array[(Int, Int, Double)]): InLinkBlock = {
-    val userIds = ratings.map(_._1).distinct.sorted
+  private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating]): InLinkBlock = {
+    val userIds = ratings.map(_.user).distinct.sorted
     val numUsers = userIds.length
     val userIdToPos = userIds.zipWithIndex.toMap
+    // Split out our ratings by product block
+    val blockRatings = Array.fill(numBlocks)(new ArrayBuffer[Rating])
+    for (r <- ratings) {
+      blockRatings(r.product % numBlocks) += r
+    }
     val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks)
     for (productBlock <- 0 until numBlocks) {
-      val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock)
-      val ratingsByProduct = ratingsInBlock.groupBy(_._2)  // (p, Seq[(u, p, r)])
-                               .toArray
-                               .sortBy(_._1)
-                               .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))}
-      ratingsForBlock(productBlock) = ratingsByProduct
+      // Create an array of (product, Seq(Rating)) ratings
+      val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray
+      // Sort them by user ID
+      val ordering = new Ordering[(Int, ArrayBuffer[Rating])] {
+        def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = a._1 - b._1
+      }
+      Sorting.quickSort(groupedRatings)(ordering)
+      // Translate the user IDs to indices based on userIdToPos
+      ratingsForBlock(productBlock) = groupedRatings.map { case (p, rs) =>
+        (rs.view.map(r => userIdToPos(r.user)).toArray, rs.view.map(_.rating).toArray)
+      }
     }
     InLinkBlock(userIds, ratingsForBlock)
   }
@@ -167,7 +184,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
   {
     val grouped = ratings.partitionBy(new HashPartitioner(numBlocks))
     val links = grouped.mapPartitionsWithIndex((blockId, elements) => {
-      val ratings = elements.map(_._2).toArray
+      val ratings = elements.map{case (k, t) => Rating(t._1, t._2, t._3)}.toArray
       val inLinkBlock = makeInLinkBlock(numBlocks, ratings)
       val outLinkBlock = makeOutLinkBlock(numBlocks, ratings)
       Iterator.single((blockId, (inLinkBlock, outLinkBlock)))
@@ -373,6 +390,8 @@ object ALS {
     }
     val (master, ratingsFile, rank, iters, outputDir) =
       (args(0), args(1), args(2).toInt, args(3).toInt, args(4))
+    System.setProperty("spark.serializer", "spark.KryoSerializer")
+    System.setProperty("spark.locality.wait", "10000")
     val sc = new SparkContext(master, "ALS")
     val ratings = sc.textFile(ratingsFile).map { line =>
       val fields = line.split(',')


[40/50] [abbrv] git commit: Update to latest Scala Maven plugin and allow Zinc external compiler

Posted by ma...@apache.org.
Update to latest Scala Maven plugin and allow Zinc external compiler


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/00a14deb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/00a14deb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/00a14deb

Branch: refs/heads/master
Commit: 00a14deb6d9b6d89e0ddbe1540636622b1dc4d16
Parents: b1f9f64
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Tue Jul 16 11:52:20 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Tue Jul 16 11:52:20 2013 -0700

----------------------------------------------------------------------
 pom.xml | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/00a14deb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 63dd80b..6fa1f1d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -319,7 +319,7 @@
         <plugin>
           <groupId>net.alchim31.maven</groupId>
           <artifactId>scala-maven-plugin</artifactId>
-          <version>3.1.0</version>
+          <version>3.1.5</version>
           <executions>
             <execution>
               <id>scala-compile-first</id>
@@ -346,12 +346,14 @@
           <configuration>
             <scalaVersion>${scala.version}</scalaVersion>
             <recompileMode>incremental</recompileMode>
+            <useZincServer>true</useZincServer>
             <args>
               <arg>-unchecked</arg>
               <arg>-optimise</arg>
             </args>
             <jvmArgs>
               <jvmArg>-Xms64m</jvmArg>
+              <jvmArg>-Xms1024m</jvmArg>
               <jvmArg>-Xmx1024m</jvmArg>
               <jvmArg>-XX:PermSize=${PermGen}</jvmArg>
               <jvmArg>-XX:MaxPermSize=${MaxPermGen}</jvmArg>


[48/50] [abbrv] git commit: Add Apache license headers and LICENSE and NOTICE files

Posted by ma...@apache.org.
Add Apache license headers and LICENSE and NOTICE files


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/af3c9d50
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/af3c9d50
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/af3c9d50

Branch: refs/heads/master
Commit: af3c9d50424602f3e5af1055e83e9badef0a1632
Parents: 00a14de
Author: Matei Zaharia <ma...@eecs.berkeley.edu>
Authored: Tue Jul 16 17:21:33 2013 -0700
Committer: Matei Zaharia <ma...@eecs.berkeley.edu>
Committed: Tue Jul 16 17:21:33 2013 -0700

----------------------------------------------------------------------
 LICENSE                                         | 229 ++++++++++++++++---
 NOTICE                                          |   5 +
 bagel/pom.xml                                   |  17 ++
 bagel/src/main/scala/spark/bagel/Bagel.scala    |  17 ++
 .../spark/bagel/examples/PageRankUtils.scala    |  17 ++
 .../bagel/examples/WikipediaPageRank.scala      |  17 ++
 .../examples/WikipediaPageRankStandalone.scala  |  17 ++
 bagel/src/test/resources/log4j.properties       |  19 +-
 bagel/src/test/scala/bagel/BagelSuite.scala     |  17 ++
 bin/slaves.sh                                   |   5 +-
 bin/spark-config.sh                             |  19 +-
 bin/spark-daemon.sh                             |   5 +-
 bin/spark-daemons.sh                            |  17 ++
 bin/start-all.sh                                |  17 ++
 bin/start-master.sh                             |  17 ++
 bin/start-slave.sh                              |  17 ++
 bin/start-slaves.sh                             |  17 ++
 bin/stop-all.sh                                 |  17 ++
 bin/stop-master.sh                              |  17 ++
 bin/stop-slaves.sh                              |  17 ++
 core/pom.xml                                    |  17 ++
 .../apache/hadoop/mapred/HadoopMapRedUtil.scala |  17 ++
 .../hadoop/mapreduce/HadoopMapReduceUtil.scala  |  17 ++
 .../scala/spark/deploy/SparkHadoopUtil.scala    |  17 ++
 .../apache/hadoop/mapred/HadoopMapRedUtil.scala |  17 ++
 .../hadoop/mapreduce/HadoopMapReduceUtil.scala  |  17 ++
 .../scala/spark/deploy/SparkHadoopUtil.scala    |  17 ++
 .../spark/deploy/yarn/ApplicationMaster.scala   |  17 ++
 .../yarn/ApplicationMasterArguments.scala       |  17 ++
 .../scala/spark/deploy/yarn/Client.scala        |  17 ++
 .../spark/deploy/yarn/ClientArguments.scala     |  17 ++
 .../spark/deploy/yarn/WorkerRunnable.scala      |  17 ++
 .../deploy/yarn/YarnAllocationHandler.scala     |  17 ++
 .../cluster/YarnClusterScheduler.scala          |  17 ++
 .../apache/hadoop/mapred/HadoopMapRedUtil.scala |  17 ++
 .../hadoop/mapreduce/HadoopMapReduceUtil.scala  |  17 ++
 .../scala/spark/deploy/SparkHadoopUtil.scala    |  17 ++
 .../java/spark/network/netty/FileClient.java    |  17 ++
 .../netty/FileClientChannelInitializer.java     |  17 ++
 .../spark/network/netty/FileClientHandler.java  |  17 ++
 .../java/spark/network/netty/FileServer.java    |  17 ++
 .../netty/FileServerChannelInitializer.java     |  17 ++
 .../spark/network/netty/FileServerHandler.java  |  17 ++
 .../java/spark/network/netty/PathResolver.java  |  17 ++
 core/src/main/scala/spark/Accumulators.scala    |  17 ++
 core/src/main/scala/spark/Aggregator.scala      |  17 ++
 .../scala/spark/BlockStoreShuffleFetcher.scala  |  17 ++
 core/src/main/scala/spark/Cache.scala           |  17 ++
 core/src/main/scala/spark/CacheManager.scala    |  17 ++
 core/src/main/scala/spark/ClosureCleaner.scala  |  17 ++
 core/src/main/scala/spark/Dependency.scala      |  17 ++
 .../main/scala/spark/DoubleRDDFunctions.scala   |  17 ++
 .../main/scala/spark/FetchFailedException.scala |  17 ++
 core/src/main/scala/spark/HadoopWriter.scala    |  17 ++
 core/src/main/scala/spark/HttpFileServer.scala  |  19 +-
 core/src/main/scala/spark/HttpServer.scala      |  17 ++
 core/src/main/scala/spark/JavaSerializer.scala  |  17 ++
 core/src/main/scala/spark/KryoSerializer.scala  |  17 ++
 core/src/main/scala/spark/Logging.scala         |  17 ++
 .../src/main/scala/spark/MapOutputTracker.scala |  17 ++
 .../src/main/scala/spark/PairRDDFunctions.scala |  17 ++
 core/src/main/scala/spark/Partition.scala       |  17 ++
 core/src/main/scala/spark/Partitioner.scala     |  17 ++
 core/src/main/scala/spark/RDD.scala             |  17 ++
 .../main/scala/spark/RDDCheckpointData.scala    |  17 ++
 .../scala/spark/SequenceFileRDDFunctions.scala  |  17 ++
 .../main/scala/spark/SerializableWritable.scala |  17 ++
 core/src/main/scala/spark/ShuffleFetcher.scala  |  17 ++
 core/src/main/scala/spark/SizeEstimator.scala   |  17 ++
 .../main/scala/spark/SoftReferenceCache.scala   |  17 ++
 core/src/main/scala/spark/SparkContext.scala    |  17 ++
 core/src/main/scala/spark/SparkEnv.scala        |  17 ++
 core/src/main/scala/spark/SparkException.scala  |  17 ++
 core/src/main/scala/spark/SparkFiles.java       |  17 ++
 core/src/main/scala/spark/TaskContext.scala     |  17 ++
 core/src/main/scala/spark/TaskEndReason.scala   |  17 ++
 core/src/main/scala/spark/TaskState.scala       |  17 ++
 core/src/main/scala/spark/Utils.scala           |  17 ++
 .../scala/spark/api/java/JavaDoubleRDD.scala    |  17 ++
 .../main/scala/spark/api/java/JavaPairRDD.scala |  17 ++
 .../src/main/scala/spark/api/java/JavaRDD.scala |  17 ++
 .../main/scala/spark/api/java/JavaRDDLike.scala |  17 ++
 .../scala/spark/api/java/JavaSparkContext.scala |  17 ++
 .../java/JavaSparkContextVarargsWorkaround.java |  17 ++
 .../scala/spark/api/java/StorageLevels.java     |  17 ++
 .../java/function/DoubleFlatMapFunction.java    |  17 ++
 .../spark/api/java/function/DoubleFunction.java |  17 ++
 .../api/java/function/FlatMapFunction.scala     |  17 ++
 .../api/java/function/FlatMapFunction2.scala    |  17 ++
 .../scala/spark/api/java/function/Function.java |  17 ++
 .../spark/api/java/function/Function2.java      |  17 ++
 .../api/java/function/PairFlatMapFunction.java  |  17 ++
 .../spark/api/java/function/PairFunction.java   |  17 ++
 .../spark/api/java/function/VoidFunction.scala  |  19 +-
 .../api/java/function/WrappedFunction1.scala    |  17 ++
 .../api/java/function/WrappedFunction2.scala    |  17 ++
 .../spark/api/python/PythonPartitioner.scala    |  17 ++
 .../main/scala/spark/api/python/PythonRDD.scala |  17 ++
 .../spark/api/python/PythonWorkerFactory.scala  |  17 ++
 .../spark/broadcast/BitTorrentBroadcast.scala   |  17 ++
 .../main/scala/spark/broadcast/Broadcast.scala  |  17 ++
 .../spark/broadcast/BroadcastFactory.scala      |  17 ++
 .../scala/spark/broadcast/HttpBroadcast.scala   |  17 ++
 .../scala/spark/broadcast/MultiTracker.scala    |  17 ++
 .../main/scala/spark/broadcast/SourceInfo.scala |  17 ++
 .../scala/spark/broadcast/TreeBroadcast.scala   |  17 ++
 .../spark/deploy/ApplicationDescription.scala   |  17 ++
 core/src/main/scala/spark/deploy/Command.scala  |  17 ++
 .../main/scala/spark/deploy/DeployMessage.scala |  17 ++
 .../main/scala/spark/deploy/ExecutorState.scala |  17 ++
 .../main/scala/spark/deploy/JsonProtocol.scala  |  19 +-
 .../scala/spark/deploy/LocalSparkCluster.scala  |  17 ++
 core/src/main/scala/spark/deploy/WebUI.scala    |  17 ++
 .../main/scala/spark/deploy/client/Client.scala |  17 ++
 .../spark/deploy/client/ClientListener.scala    |  17 ++
 .../scala/spark/deploy/client/TestClient.scala  |  17 ++
 .../spark/deploy/client/TestExecutor.scala      |  17 ++
 .../spark/deploy/master/ApplicationInfo.scala   |  17 ++
 .../spark/deploy/master/ApplicationState.scala  |  17 ++
 .../spark/deploy/master/ExecutorInfo.scala      |  17 ++
 .../main/scala/spark/deploy/master/Master.scala |  17 ++
 .../spark/deploy/master/MasterArguments.scala   |  17 ++
 .../scala/spark/deploy/master/WorkerInfo.scala  |  17 ++
 .../scala/spark/deploy/master/WorkerState.scala |  17 ++
 .../deploy/master/ui/ApplicationPage.scala      |  17 ++
 .../spark/deploy/master/ui/IndexPage.scala      |  17 ++
 .../spark/deploy/master/ui/MasterWebUI.scala    |  19 +-
 .../spark/deploy/worker/ExecutorRunner.scala    |  17 ++
 .../main/scala/spark/deploy/worker/Worker.scala |  17 ++
 .../spark/deploy/worker/WorkerArguments.scala   |  17 ++
 .../spark/deploy/worker/ui/IndexPage.scala      |  17 ++
 .../spark/deploy/worker/ui/WorkerWebUI.scala    |  17 ++
 .../main/scala/spark/executor/Executor.scala    |  17 ++
 .../scala/spark/executor/ExecutorBackend.scala  |  17 ++
 .../scala/spark/executor/ExecutorExitCode.scala |  17 ++
 .../spark/executor/ExecutorURLClassLoader.scala |  17 ++
 .../spark/executor/MesosExecutorBackend.scala   |  17 ++
 .../executor/StandaloneExecutorBackend.scala    |  17 ++
 .../main/scala/spark/executor/TaskMetrics.scala |  17 ++
 .../scala/spark/network/BufferMessage.scala     |  19 +-
 .../main/scala/spark/network/Connection.scala   |  17 ++
 .../scala/spark/network/ConnectionManager.scala |  17 ++
 .../spark/network/ConnectionManagerId.scala     |  17 ++
 .../spark/network/ConnectionManagerTest.scala   |  17 ++
 core/src/main/scala/spark/network/Message.scala |  17 ++
 .../main/scala/spark/network/MessageChunk.scala |  17 ++
 .../spark/network/MessageChunkHeader.scala      |  17 ++
 .../main/scala/spark/network/ReceiverTest.scala |  17 ++
 .../main/scala/spark/network/SenderTest.scala   |  17 ++
 .../scala/spark/network/netty/FileHeader.scala  |  17 ++
 .../spark/network/netty/ShuffleCopier.scala     |  17 ++
 .../spark/network/netty/ShuffleSender.scala     |  17 ++
 core/src/main/scala/spark/package.scala         |  17 ++
 .../partial/ApproximateActionListener.scala     |  17 ++
 .../spark/partial/ApproximateEvaluator.scala    |  17 ++
 .../scala/spark/partial/BoundedDouble.scala     |  17 ++
 .../scala/spark/partial/CountEvaluator.scala    |  17 ++
 .../spark/partial/GroupedCountEvaluator.scala   |  17 ++
 .../spark/partial/GroupedMeanEvaluator.scala    |  17 ++
 .../spark/partial/GroupedSumEvaluator.scala     |  17 ++
 .../scala/spark/partial/MeanEvaluator.scala     |  17 ++
 .../scala/spark/partial/PartialResult.scala     |  17 ++
 .../scala/spark/partial/StudentTCacher.scala    |  17 ++
 .../main/scala/spark/partial/SumEvaluator.scala |  17 ++
 core/src/main/scala/spark/rdd/BlockRDD.scala    |  17 ++
 .../src/main/scala/spark/rdd/CartesianRDD.scala |  17 ++
 .../main/scala/spark/rdd/CheckpointRDD.scala    |  17 ++
 .../src/main/scala/spark/rdd/CoGroupedRDD.scala |  17 ++
 .../src/main/scala/spark/rdd/CoalescedRDD.scala |  17 ++
 core/src/main/scala/spark/rdd/EmptyRDD.scala    |  17 ++
 core/src/main/scala/spark/rdd/FilteredRDD.scala |  17 ++
 .../main/scala/spark/rdd/FlatMappedRDD.scala    |  17 ++
 core/src/main/scala/spark/rdd/GlommedRDD.scala  |  17 ++
 core/src/main/scala/spark/rdd/HadoopRDD.scala   |  17 ++
 core/src/main/scala/spark/rdd/JdbcRDD.scala     |  17 ++
 .../main/scala/spark/rdd/MapPartitionsRDD.scala |  17 ++
 .../spark/rdd/MapPartitionsWithIndexRDD.scala   |  17 ++
 core/src/main/scala/spark/rdd/MappedRDD.scala   |  17 ++
 .../src/main/scala/spark/rdd/NewHadoopRDD.scala |  17 ++
 .../scala/spark/rdd/ParallelCollectionRDD.scala |  17 ++
 .../scala/spark/rdd/PartitionPruningRDD.scala   |  17 ++
 core/src/main/scala/spark/rdd/PipedRDD.scala    |  17 ++
 core/src/main/scala/spark/rdd/SampledRDD.scala  |  17 ++
 core/src/main/scala/spark/rdd/ShuffledRDD.scala |  17 ++
 .../main/scala/spark/rdd/SubtractedRDD.scala    |  17 ++
 core/src/main/scala/spark/rdd/UnionRDD.scala    |  17 ++
 .../scala/spark/rdd/ZippedPartitionsRDD.scala   |  17 ++
 core/src/main/scala/spark/rdd/ZippedRDD.scala   |  17 ++
 .../main/scala/spark/scheduler/ActiveJob.scala  |  17 ++
 .../scala/spark/scheduler/DAGScheduler.scala    |  17 ++
 .../spark/scheduler/DAGSchedulerEvent.scala     |  17 ++
 .../scala/spark/scheduler/InputFormatInfo.scala |  17 ++
 .../scala/spark/scheduler/JobListener.scala     |  17 ++
 .../main/scala/spark/scheduler/JobLogger.scala  |  17 ++
 .../main/scala/spark/scheduler/JobResult.scala  |  17 ++
 .../main/scala/spark/scheduler/JobWaiter.scala  |  17 ++
 .../main/scala/spark/scheduler/MapStatus.scala  |  17 ++
 .../main/scala/spark/scheduler/ResultTask.scala |  17 ++
 .../scala/spark/scheduler/ShuffleMapTask.scala  |  17 ++
 .../scala/spark/scheduler/SparkListener.scala   |  17 ++
 .../main/scala/spark/scheduler/SplitInfo.scala  |  17 ++
 core/src/main/scala/spark/scheduler/Stage.scala |  17 ++
 .../main/scala/spark/scheduler/StageInfo.scala  |  19 +-
 core/src/main/scala/spark/scheduler/Task.scala  |  17 ++
 .../main/scala/spark/scheduler/TaskResult.scala |  17 ++
 .../scala/spark/scheduler/TaskScheduler.scala   |  17 ++
 .../spark/scheduler/TaskSchedulerListener.scala |  17 ++
 .../main/scala/spark/scheduler/TaskSet.scala    |  17 ++
 .../scheduler/cluster/ClusterScheduler.scala    |  17 ++
 .../cluster/ClusterTaskSetManager.scala         |  17 ++
 .../scheduler/cluster/ExecutorLossReason.scala  |  17 ++
 .../scala/spark/scheduler/cluster/Pool.scala    |  17 ++
 .../spark/scheduler/cluster/Schedulable.scala   |  17 ++
 .../scheduler/cluster/SchedulableBuilder.scala  |  17 ++
 .../scheduler/cluster/SchedulerBackend.scala    |  17 ++
 .../scheduler/cluster/SchedulingAlgorithm.scala |  17 ++
 .../scheduler/cluster/SchedulingMode.scala      |  17 ++
 .../cluster/SparkDeploySchedulerBackend.scala   |  17 ++
 .../cluster/StandaloneClusterMessage.scala      |  17 ++
 .../cluster/StandaloneSchedulerBackend.scala    |  17 ++
 .../scheduler/cluster/TaskDescription.scala     |  17 ++
 .../spark/scheduler/cluster/TaskInfo.scala      |  17 ++
 .../scheduler/cluster/TaskSetManager.scala      |  17 ++
 .../spark/scheduler/cluster/WorkerOffer.scala   |  17 ++
 .../spark/scheduler/local/LocalScheduler.scala  |  17 ++
 .../scheduler/local/LocalTaskSetManager.scala   |  17 ++
 .../mesos/CoarseMesosSchedulerBackend.scala     |  17 ++
 .../scheduler/mesos/MesosSchedulerBackend.scala |  17 ++
 .../scala/spark/serializer/Serializer.scala     |  17 ++
 .../spark/serializer/SerializerManager.scala    |  17 ++
 .../scala/spark/storage/BlockException.scala    |  17 ++
 .../scala/spark/storage/BlockFetchTracker.scala |  17 ++
 .../spark/storage/BlockFetcherIterator.scala    |  17 ++
 .../main/scala/spark/storage/BlockManager.scala |  17 ++
 .../scala/spark/storage/BlockManagerId.scala    |  17 ++
 .../spark/storage/BlockManagerMaster.scala      |  17 ++
 .../spark/storage/BlockManagerMasterActor.scala |  17 ++
 .../spark/storage/BlockManagerMessages.scala    |  17 ++
 .../spark/storage/BlockManagerSlaveActor.scala  |  17 ++
 .../spark/storage/BlockManagerWorker.scala      |  17 ++
 .../main/scala/spark/storage/BlockMessage.scala |  17 ++
 .../scala/spark/storage/BlockMessageArray.scala |  17 ++
 .../scala/spark/storage/BlockObjectWriter.scala |  17 ++
 .../main/scala/spark/storage/BlockStore.scala   |  17 ++
 .../main/scala/spark/storage/DiskStore.scala    |  17 ++
 .../main/scala/spark/storage/MemoryStore.scala  |  17 ++
 .../main/scala/spark/storage/PutResult.scala    |  17 ++
 .../spark/storage/ShuffleBlockManager.scala     |  17 ++
 .../main/scala/spark/storage/StorageLevel.scala |  17 ++
 .../main/scala/spark/storage/StorageUtils.scala |  17 ++
 .../scala/spark/storage/ThreadingTest.scala     |  17 ++
 core/src/main/scala/spark/ui/JettyUtils.scala   |  17 ++
 core/src/main/scala/spark/ui/Page.scala         |  19 +-
 core/src/main/scala/spark/ui/SparkUI.scala      |  17 ++
 core/src/main/scala/spark/ui/UIUtils.scala      |  19 +-
 .../scala/spark/ui/UIWorkloadGenerator.scala    |  17 ++
 .../main/scala/spark/ui/env/EnvironmentUI.scala |  17 ++
 .../main/scala/spark/ui/jobs/IndexPage.scala    |  17 ++
 .../scala/spark/ui/jobs/JobProgressUI.scala     |  19 +-
 .../main/scala/spark/ui/jobs/StagePage.scala    |  17 ++
 .../scala/spark/ui/storage/BlockManagerUI.scala |  17 ++
 .../main/scala/spark/ui/storage/IndexPage.scala |  17 ++
 .../main/scala/spark/ui/storage/RDDPage.scala   |  17 ++
 core/src/main/scala/spark/util/AkkaUtils.scala  |  17 ++
 .../scala/spark/util/BoundedPriorityQueue.scala |  17 ++
 .../spark/util/ByteBufferInputStream.scala      |  17 ++
 .../scala/spark/util/CompletionIterator.scala   |  19 +-
 .../main/scala/spark/util/Distribution.scala    |  19 +-
 .../src/main/scala/spark/util/IdGenerator.scala |  17 ++
 core/src/main/scala/spark/util/IntParam.scala   |  17 ++
 .../src/main/scala/spark/util/MemoryParam.scala |  17 ++
 .../main/scala/spark/util/MetadataCleaner.scala |  17 ++
 .../main/scala/spark/util/NextIterator.scala    |  19 +-
 .../spark/util/RateLimitedOutputStream.scala    |  17 ++
 .../scala/spark/util/SerializableBuffer.scala   |  17 ++
 .../src/main/scala/spark/util/StatCounter.scala |  17 ++
 .../scala/spark/util/TimeStampedHashMap.scala   |  17 ++
 .../scala/spark/util/TimeStampedHashSet.scala   |  17 ++
 core/src/main/scala/spark/util/Vector.scala     |  17 ++
 core/src/test/resources/fairscheduler.xml       |  17 ++
 core/src/test/resources/log4j.properties        |  19 +-
 .../src/test/scala/spark/AccumulatorSuite.scala |  17 ++
 core/src/test/scala/spark/BroadcastSuite.scala  |  17 ++
 core/src/test/scala/spark/CheckpointSuite.scala |  17 ++
 .../test/scala/spark/ClosureCleanerSuite.scala  |  17 ++
 .../src/test/scala/spark/DistributedSuite.scala |  17 ++
 core/src/test/scala/spark/DriverSuite.scala     |  17 ++
 core/src/test/scala/spark/FailureSuite.scala    |  17 ++
 core/src/test/scala/spark/FileServerSuite.scala |  17 ++
 core/src/test/scala/spark/FileSuite.scala       |  17 ++
 core/src/test/scala/spark/JavaAPISuite.java     |  17 ++
 .../test/scala/spark/KryoSerializerSuite.scala  |  17 ++
 .../test/scala/spark/LocalSparkContext.scala    |  17 ++
 .../scala/spark/MapOutputTrackerSuite.scala     |  17 ++
 .../scala/spark/PairRDDFunctionsSuite.scala     |  17 ++
 .../test/scala/spark/PartitioningSuite.scala    |  17 ++
 core/src/test/scala/spark/PipedRDDSuite.scala   |  17 ++
 core/src/test/scala/spark/RDDSuite.scala        |  17 ++
 .../test/scala/spark/SharedSparkContext.scala   |  17 ++
 .../test/scala/spark/ShuffleNettySuite.scala    |  17 ++
 core/src/test/scala/spark/ShuffleSuite.scala    |  17 ++
 .../test/scala/spark/SizeEstimatorSuite.scala   |  17 ++
 core/src/test/scala/spark/SortingSuite.scala    |  17 ++
 core/src/test/scala/spark/ThreadingSuite.scala  |  17 ++
 core/src/test/scala/spark/UnpersistSuite.scala  |  17 ++
 core/src/test/scala/spark/UtilsSuite.scala      |  17 ++
 .../scala/spark/ZippedPartitionsSuite.scala     |  17 ++
 .../src/test/scala/spark/rdd/JdbcRDDSuite.scala |  17 ++
 .../rdd/ParallelCollectionSplitSuite.scala      |  17 ++
 .../spark/scheduler/ClusterSchedulerSuite.scala |  17 ++
 .../spark/scheduler/DAGSchedulerSuite.scala     |  17 ++
 .../scala/spark/scheduler/JobLoggerSuite.scala  |  17 ++
 .../spark/scheduler/LocalSchedulerSuite.scala   |  17 ++
 .../spark/scheduler/SparkListenerSuite.scala    |  17 ++
 .../spark/scheduler/TaskContextSuite.scala      |  17 ++
 .../scala/spark/storage/BlockManagerSuite.scala |  17 ++
 core/src/test/scala/spark/ui/UISuite.scala      |  17 ++
 .../scala/spark/util/DistributionSuite.scala    |  17 ++
 .../scala/spark/util/NextIteratorSuite.scala    |  17 ++
 .../util/RateLimitedOutputStreamSuite.scala     |  17 ++
 docs/_plugins/copy_api_dirs.rb                  |  17 ++
 ec2/spark-ec2                                   |   2 +
 ec2/spark_ec2.py                                |   2 +
 examples/pom.xml                                |  17 ++
 .../main/java/spark/examples/JavaHdfsLR.java    |  17 ++
 .../main/java/spark/examples/JavaKMeans.java    |  17 ++
 .../main/java/spark/examples/JavaLogQuery.java  |  17 ++
 .../main/java/spark/examples/JavaSparkPi.java   |  17 ++
 .../src/main/java/spark/examples/JavaTC.java    |  17 ++
 .../main/java/spark/examples/JavaWordCount.java |  17 ++
 .../streaming/examples/JavaFlumeEventCount.java |  17 ++
 .../examples/JavaNetworkWordCount.java          |  17 ++
 .../streaming/examples/JavaQueueStream.java     |  17 ++
 .../scala/spark/examples/BroadcastTest.scala    |  17 ++
 .../scala/spark/examples/CassandraTest.scala    |  17 ++
 .../spark/examples/ExceptionHandlingTest.scala  |  17 ++
 .../main/scala/spark/examples/GroupByTest.scala |  17 ++
 .../main/scala/spark/examples/HBaseTest.scala   |  19 +-
 .../main/scala/spark/examples/HdfsTest.scala    |  17 ++
 .../main/scala/spark/examples/LocalALS.scala    |  17 ++
 .../main/scala/spark/examples/LocalFileLR.scala |  17 ++
 .../main/scala/spark/examples/LocalKMeans.scala |  17 ++
 .../src/main/scala/spark/examples/LocalLR.scala |  17 ++
 .../src/main/scala/spark/examples/LocalPi.scala |  17 ++
 .../main/scala/spark/examples/LogQuery.scala    |  17 ++
 .../spark/examples/MultiBroadcastTest.scala     |  17 ++
 .../examples/SimpleSkewedGroupByTest.scala      |  17 ++
 .../spark/examples/SkewedGroupByTest.scala      |  17 ++
 .../main/scala/spark/examples/SparkALS.scala    |  17 ++
 .../main/scala/spark/examples/SparkHdfsLR.scala |  17 ++
 .../main/scala/spark/examples/SparkKMeans.scala |  17 ++
 .../src/main/scala/spark/examples/SparkLR.scala |  17 ++
 .../src/main/scala/spark/examples/SparkPi.scala |  17 ++
 .../src/main/scala/spark/examples/SparkTC.scala |  17 ++
 .../streaming/examples/ActorWordCount.scala     |  17 ++
 .../streaming/examples/FlumeEventCount.scala    |  17 ++
 .../streaming/examples/HdfsWordCount.scala      |  17 ++
 .../streaming/examples/KafkaWordCount.scala     |  17 ++
 .../streaming/examples/NetworkWordCount.scala   |  17 ++
 .../spark/streaming/examples/QueueStream.scala  |  17 ++
 .../streaming/examples/RawNetworkGrep.scala     |  17 ++
 .../examples/StatefulNetworkWordCount.scala     |  17 ++
 .../streaming/examples/TwitterAlgebirdCMS.scala |  17 ++
 .../streaming/examples/TwitterAlgebirdHLL.scala |  17 ++
 .../streaming/examples/TwitterPopularTags.scala |  17 ++
 .../streaming/examples/ZeroMQWordCount.scala    |  17 ++
 .../clickstream/PageViewGenerator.scala         |  17 ++
 .../examples/clickstream/PageViewStream.scala   |  17 ++
 make-distribution.sh                            |  19 ++
 .../scala/spark/mllib/clustering/KMeans.scala   |  17 ++
 .../spark/mllib/clustering/KMeansModel.scala    |  17 ++
 .../spark/mllib/clustering/LocalKMeans.scala    |  17 ++
 .../spark/mllib/optimization/Gradient.scala     |  17 ++
 .../mllib/optimization/GradientDescent.scala    |  17 ++
 .../spark/mllib/optimization/Updater.scala      |  17 ++
 .../scala/spark/mllib/recommendation/ALS.scala  |  17 ++
 .../MatrixFactorizationModel.scala              |  17 ++
 .../mllib/regression/LogisticRegression.scala   |  17 ++
 .../LogisticRegressionGenerator.scala           |  17 ++
 .../spark/mllib/regression/Regression.scala     |  17 ++
 .../mllib/regression/RidgeRegression.scala      |  17 ++
 .../regression/RidgeRegressionGenerator.scala   |  17 ++
 .../main/scala/spark/mllib/util/MLUtils.scala   |  17 ++
 mllib/src/test/resources/log4j.properties       |  19 +-
 .../spark/mllib/clustering/KMeansSuite.scala    |  17 ++
 .../spark/mllib/recommendation/ALSSuite.scala   |  17 ++
 .../regression/LogisticRegressionSuite.scala    |  17 ++
 .../mllib/regression/RidgeRegressionSuite.scala |  17 ++
 pom.xml                                         |  17 ++
 project/SparkBuild.scala                        |  16 ++
 project/build.properties                        |  17 ++
 pyspark                                         |  17 ++
 python/epydoc.conf                              |  17 ++
 python/examples/als.py                          |  17 ++
 python/examples/kmeans.py                       |  17 ++
 python/examples/logistic_regression.py          |  17 ++
 python/examples/pi.py                           |  17 ++
 python/examples/transitive_closure.py           |  17 ++
 python/examples/wordcount.py                    |  17 ++
 python/pyspark/accumulators.py                  |  17 ++
 python/pyspark/broadcast.py                     |  17 ++
 python/pyspark/context.py                       |  17 ++
 python/pyspark/daemon.py                        |  17 ++
 python/pyspark/files.py                         |  17 ++
 python/pyspark/java_gateway.py                  |  17 ++
 python/pyspark/rdd.py                           |  17 ++
 python/pyspark/serializers.py                   |  17 ++
 python/pyspark/shell.py                         |  17 ++
 python/pyspark/tests.py                         |  17 ++
 python/pyspark/worker.py                        |  17 ++
 python/run-tests                                |  20 +-
 repl-bin/pom.xml                                |  17 ++
 repl-bin/src/deb/bin/run                        |  19 +-
 repl-bin/src/deb/bin/spark-executor             |  17 ++
 repl-bin/src/deb/bin/spark-shell                |  17 ++
 repl/pom.xml                                    |  17 ++
 .../scala/spark/repl/ExecutorClassLoader.scala  |  17 ++
 repl/src/main/scala/spark/repl/Main.scala       |  17 ++
 repl/src/test/resources/log4j.properties        |  19 +-
 repl/src/test/scala/spark/repl/ReplSuite.scala  |  17 ++
 run                                             |  17 ++
 run.cmd                                         |  18 ++
 run2.cmd                                        |  17 ++
 sbt/sbt                                         |  20 ++
 sbt/sbt.cmd                                     |  20 ++
 spark-executor                                  |  18 ++
 spark-shell                                     |  19 ++
 spark-shell.cmd                                 |  18 ++
 streaming/pom.xml                               |  17 ++
 .../main/scala/spark/streaming/Checkpoint.scala |  17 ++
 .../main/scala/spark/streaming/DStream.scala    |  17 ++
 .../spark/streaming/DStreamCheckpointData.scala |  17 ++
 .../scala/spark/streaming/DStreamGraph.scala    |  17 ++
 .../main/scala/spark/streaming/Duration.scala   |  17 ++
 .../main/scala/spark/streaming/Interval.scala   |  17 ++
 .../src/main/scala/spark/streaming/Job.scala    |  17 ++
 .../main/scala/spark/streaming/JobManager.scala |  17 ++
 .../spark/streaming/NetworkInputTracker.scala   |  17 ++
 .../spark/streaming/PairDStreamFunctions.scala  |  17 ++
 .../main/scala/spark/streaming/Scheduler.scala  |  17 ++
 .../spark/streaming/StreamingContext.scala      |  17 ++
 .../src/main/scala/spark/streaming/Time.scala   |  19 +-
 .../spark/streaming/api/java/JavaDStream.scala  |  19 +-
 .../streaming/api/java/JavaDStreamLike.scala    |  19 +-
 .../streaming/api/java/JavaPairDStream.scala    |  17 ++
 .../api/java/JavaStreamingContext.scala         |  17 ++
 .../streaming/dstream/CoGroupedDStream.scala    |  17 ++
 .../dstream/ConstantInputDStream.scala          |  19 +-
 .../streaming/dstream/FileInputDStream.scala    |  17 ++
 .../streaming/dstream/FilteredDStream.scala     |  17 ++
 .../dstream/FlatMapValuedDStream.scala          |  17 ++
 .../streaming/dstream/FlatMappedDStream.scala   |  17 ++
 .../streaming/dstream/FlumeInputDStream.scala   |  17 ++
 .../streaming/dstream/ForEachDStream.scala      |  17 ++
 .../streaming/dstream/GlommedDStream.scala      |  17 ++
 .../spark/streaming/dstream/InputDStream.scala  |  17 ++
 .../streaming/dstream/KafkaInputDStream.scala   |  17 ++
 .../dstream/MapPartitionedDStream.scala         |  17 ++
 .../streaming/dstream/MapValuedDStream.scala    |  17 ++
 .../spark/streaming/dstream/MappedDStream.scala |  17 ++
 .../streaming/dstream/NetworkInputDStream.scala |  17 ++
 .../dstream/PluggableInputDStream.scala         |  17 ++
 .../streaming/dstream/QueueInputDStream.scala   |  17 ++
 .../streaming/dstream/RawInputDStream.scala     |  17 ++
 .../dstream/ReducedWindowedDStream.scala        |  17 ++
 .../streaming/dstream/ShuffledDStream.scala     |  17 ++
 .../streaming/dstream/SocketInputDStream.scala  |  17 ++
 .../spark/streaming/dstream/StateDStream.scala  |  17 ++
 .../streaming/dstream/TransformedDStream.scala  |  17 ++
 .../streaming/dstream/TwitterInputDStream.scala |  17 ++
 .../spark/streaming/dstream/UnionDStream.scala  |  17 ++
 .../streaming/dstream/WindowedDStream.scala     |  17 ++
 .../streaming/receivers/ActorReceiver.scala     |  17 ++
 .../streaming/receivers/ZeroMQReceiver.scala    |  17 ++
 .../main/scala/spark/streaming/util/Clock.scala |  17 ++
 .../streaming/util/MasterFailureTest.scala      |  17 ++
 .../spark/streaming/util/RawTextHelper.scala    |  17 ++
 .../spark/streaming/util/RawTextSender.scala    |  17 ++
 .../spark/streaming/util/RecurringTimer.scala   |  17 ++
 .../test/java/spark/streaming/JavaAPISuite.java |  17 ++
 .../java/spark/streaming/JavaTestUtils.scala    |  19 +-
 streaming/src/test/resources/log4j.properties   |  19 +-
 .../spark/streaming/BasicOperationsSuite.scala  |  17 ++
 .../scala/spark/streaming/CheckpointSuite.scala |  17 ++
 .../scala/spark/streaming/FailureSuite.scala    |  17 ++
 .../spark/streaming/InputStreamsSuite.scala     |  17 ++
 .../scala/spark/streaming/TestSuiteBase.scala   |  17 ++
 .../spark/streaming/WindowOperationsSuite.scala |  17 ++
 488 files changed, 8448 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index d17afa1..d645695 100644
--- a/LICENSE
+++ b/LICENSE
@@ -1,27 +1,202 @@
-Copyright (c) 2010, Regents of the University of California.
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions
-are met:
-    * Redistributions of source code must retain the above copyright
-      notice, this list of conditions and the following disclaimer.
-    * Redistributions in binary form must reproduce the above copyright
-      notice, this list of conditions and the following disclaimer in the
-      documentation and/or other materials provided with the distribution.
-    * Neither the name of the University of California, Berkeley nor the
-      names of its contributors may be used to endorse or promote
-      products derived from this software without specific prior written
-      permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED
-TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
-PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
-LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
-NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
-SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
new file mode 100644
index 0000000..7cbb114
--- /dev/null
+++ b/NOTICE
@@ -0,0 +1,5 @@
+Apache Spark
+Copyright 2013 The Apache Software Foundation.
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bagel/pom.xml
----------------------------------------------------------------------
diff --git a/bagel/pom.xml b/bagel/pom.xml
index b83a0ef..60bbc49 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
 <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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bagel/src/main/scala/spark/bagel/Bagel.scala
----------------------------------------------------------------------
diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/spark/bagel/Bagel.scala
index 5ecdd7d..80c8d53 100644
--- a/bagel/src/main/scala/spark/bagel/Bagel.scala
+++ b/bagel/src/main/scala/spark/bagel/Bagel.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.bagel
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala
----------------------------------------------------------------------
diff --git a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala
index b97d786..de65e27 100644
--- a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala
+++ b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.bagel.examples
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
----------------------------------------------------------------------
diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
index bc32663..a0c5ac9 100644
--- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
+++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.bagel.examples
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
----------------------------------------------------------------------
diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
index 9d9d80d..3c54a85 100644
--- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
+++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.bagel.examples
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bagel/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties
index 83d05ca..5cdcf35 100644
--- a/bagel/src/test/resources/log4j.properties
+++ b/bagel/src/test/resources/log4j.properties
@@ -1,4 +1,21 @@
-# Set everything to be logged to the file bagel/target/unit-tests.log 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file bagel/target/unit-tests.log
 log4j.rootCategory=INFO, file
 log4j.appender.file=org.apache.log4j.FileAppender
 log4j.appender.file.append=false

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bagel/src/test/scala/bagel/BagelSuite.scala
----------------------------------------------------------------------
diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala
index a09c978..ef2d57f 100644
--- a/bagel/src/test/scala/bagel/BagelSuite.scala
+++ b/bagel/src/test/scala/bagel/BagelSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.bagel
 
 import org.scalatest.{FunSuite, Assertions, BeforeAndAfter}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/slaves.sh
----------------------------------------------------------------------
diff --git a/bin/slaves.sh b/bin/slaves.sh
index 030581e..c8fb5ca 100755
--- a/bin/slaves.sh
+++ b/bin/slaves.sh
@@ -1,7 +1,5 @@
 #!/usr/bin/env bash
 
-# This Spark deploy script is a modified version of the Apache Hadoop deploy
-# script, available under the Apache 2 license:
 #
 # Licensed to the Apache Software Foundation (ASF) under one or more
 # contributor license agreements.  See the NOTICE file distributed with
@@ -10,13 +8,14 @@
 # (the "License"); you may not use this file except in compliance with
 # the License.  You may obtain a copy of the License at
 #
-#     http://www.apache.org/licenses/LICENSE-2.0
+#    http://www.apache.org/licenses/LICENSE-2.0
 #
 # Unless required by applicable law or agreed to in writing, software
 # distributed under the License is distributed on an "AS IS" BASIS,
 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 # See the License for the specific language governing permissions and
 # limitations under the License.
+#
 
 # Run a shell command on all slave hosts.
 #

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/spark-config.sh
----------------------------------------------------------------------
diff --git a/bin/spark-config.sh b/bin/spark-config.sh
index d4b6558..cd2c7b7 100755
--- a/bin/spark-config.sh
+++ b/bin/spark-config.sh
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # included in all the spark scripts with source command
 # should not be executable directly
 # also should not be passed any arguments, since we need original $*
@@ -16,4 +33,4 @@ this="$config_bin/$script"
 
 export SPARK_PREFIX=`dirname "$this"`/..
 export SPARK_HOME=${SPARK_PREFIX}
-export SPARK_CONF_DIR="$SPARK_HOME/conf"
\ No newline at end of file
+export SPARK_CONF_DIR="$SPARK_HOME/conf"

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/spark-daemon.sh
----------------------------------------------------------------------
diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh
index 8ee3ec4..a5b88ca 100755
--- a/bin/spark-daemon.sh
+++ b/bin/spark-daemon.sh
@@ -1,7 +1,5 @@
 #!/usr/bin/env bash
 
-# This Spark deploy script is a modified version of the Apache Hadoop deploy
-# script, available under the Apache 2 license:
 #
 # Licensed to the Apache Software Foundation (ASF) under one or more
 # contributor license agreements.  See the NOTICE file distributed with
@@ -10,13 +8,14 @@
 # (the "License"); you may not use this file except in compliance with
 # the License.  You may obtain a copy of the License at
 #
-#     http://www.apache.org/licenses/LICENSE-2.0
+#    http://www.apache.org/licenses/LICENSE-2.0
 #
 # Unless required by applicable law or agreed to in writing, software
 # distributed under the License is distributed on an "AS IS" BASIS,
 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 # See the License for the specific language governing permissions and
 # limitations under the License.
+#
 
 # Runs a Spark command as a daemon.
 #

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/spark-daemons.sh
----------------------------------------------------------------------
diff --git a/bin/spark-daemons.sh b/bin/spark-daemons.sh
index 0619097..354eb90 100755
--- a/bin/spark-daemons.sh
+++ b/bin/spark-daemons.sh
@@ -1,5 +1,22 @@
 #!/usr/bin/env bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Run a Spark command on all slave hosts.
 
 usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..."

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/start-all.sh
----------------------------------------------------------------------
diff --git a/bin/start-all.sh b/bin/start-all.sh
index b9891ad..0182f1a 100755
--- a/bin/start-all.sh
+++ b/bin/start-all.sh
@@ -1,5 +1,22 @@
 #!/usr/bin/env bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Start all spark daemons.
 # Starts the master on this node.
 # Starts a worker on each node specified in conf/slaves

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/start-master.sh
----------------------------------------------------------------------
diff --git a/bin/start-master.sh b/bin/start-master.sh
index 83a3e1f..2288fb1 100755
--- a/bin/start-master.sh
+++ b/bin/start-master.sh
@@ -1,5 +1,22 @@
 #!/usr/bin/env bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Starts the master on the machine this script is executed on.
 
 bin=`dirname "$0"`

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/start-slave.sh
----------------------------------------------------------------------
diff --git a/bin/start-slave.sh b/bin/start-slave.sh
index 1082c09..d6db168 100755
--- a/bin/start-slave.sh
+++ b/bin/start-slave.sh
@@ -1,5 +1,22 @@
 #!/usr/bin/env bash
+
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
 #
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Usage: start-slave.sh <worker#> <master-spark-URL>
 #   where <master-spark-URL> is like "spark://localhost:7077"
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/start-slaves.sh
----------------------------------------------------------------------
diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh
index 4e05224..dad7c3d 100755
--- a/bin/start-slaves.sh
+++ b/bin/start-slaves.sh
@@ -1,5 +1,22 @@
 #!/usr/bin/env bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/stop-all.sh
----------------------------------------------------------------------
diff --git a/bin/stop-all.sh b/bin/stop-all.sh
index d352f6f..a043ac0 100755
--- a/bin/stop-all.sh
+++ b/bin/stop-all.sh
@@ -1,5 +1,22 @@
 #!/usr/bin/env bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Start all spark daemons.
 # Run this on the master nde
 

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/stop-master.sh
----------------------------------------------------------------------
diff --git a/bin/stop-master.sh b/bin/stop-master.sh
index 172ee58..31a610b 100755
--- a/bin/stop-master.sh
+++ b/bin/stop-master.sh
@@ -1,5 +1,22 @@
 #!/usr/bin/env bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Starts the master on the machine this script is executed on.
 
 bin=`dirname "$0"`

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/bin/stop-slaves.sh
----------------------------------------------------------------------
diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh
index fbfc594..8e056f2 100755
--- a/bin/stop-slaves.sh
+++ b/bin/stop-slaves.sh
@@ -1,5 +1,22 @@
 #!/usr/bin/env bash
 
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
 # Starts the master on the machine this script is executed on.
 
 bin=`dirname "$0"`

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index dbb2da9..6329b2f 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -1,4 +1,21 @@
 <?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one or more
+  ~ contributor license agreements.  See the NOTICE file distributed with
+  ~ this work for additional information regarding copyright ownership.
+  ~ The ASF licenses this file to You under the Apache License, Version 2.0
+  ~ (the "License"); you may not use this file except in compliance with
+  ~ the License.  You may obtain a copy of the License at
+  ~
+  ~    http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing, software
+  ~ distributed under the License is distributed on an "AS IS" BASIS,
+  ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  ~ See the License for the specific language governing permissions and
+  ~ limitations under the License.
+  -->
+
 <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>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
index f286f2c..25386b2 100644
--- a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
+++ b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.mapred
 
 trait HadoopMapRedUtil {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
index 264d421..b1002e0 100644
--- a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
+++ b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.mapreduce
 
 import org.apache.hadoop.conf.Configuration

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala
index f1c86de..df55be1 100644
--- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapred.JobConf

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
index 875c0a2..0f972b7 100644
--- a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
+++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
@@ -1,4 +1,21 @@
 
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.mapred
 
 import org.apache.hadoop.mapreduce.TaskType

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
index 8bc6fb6..1a7cdf4 100644
--- a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
+++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.mapreduce
 
 import org.apache.hadoop.conf.Configuration

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala
index 301a57f..6122fdc 100644
--- a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 
 import collection.mutable.HashMap

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
index 6a0617c..1b06169 100644
--- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.yarn
 
 import java.net.Socket

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala
index 1b00208..8de44b1 100644
--- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.yarn
 
 import spark.util.IntParam

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala
index 514c17f..8bcbfc2 100644
--- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.yarn
 
 import java.net.{InetSocketAddress, URI}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala
index 07e7ede..67aff03 100644
--- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.yarn
 
 import spark.util.MemoryParam

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala
index cc6f334..f458f2f 100644
--- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.yarn
 
 import java.net.URI

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala
index 61dd72a..b0af8ba 100644
--- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.yarn
 
 import spark.{Logging, Utils}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala
index ed732d3..307d961 100644
--- a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala
+++ b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
index a0652d7..4b3d846 100644
--- a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
+++ b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.mapred
 
 trait HadoopMapRedUtil {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
----------------------------------------------------------------------
diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
index 7fdbe32..aa3b1ed 100644
--- a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
+++ b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.mapreduce
 
 import org.apache.hadoop.conf.Configuration


[28/50] [abbrv] git commit: Links to job UI from standalone deploy cluster web UI: fixes SPARK-802

Posted by ma...@apache.org.
Links to job UI from standalone deploy cluster web UI: fixes SPARK-802


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/0d78b6d9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/0d78b6d9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/0d78b6d9

Branch: refs/heads/master
Commit: 0d78b6d9cd11fc12c546f25fa857ba8b285c062d
Parents: 4698a0d
Author: Karen Feng <ka...@gmail.com>
Authored: Mon Jul 15 13:42:12 2013 -0700
Committer: Karen Feng <ka...@gmail.com>
Committed: Mon Jul 15 13:47:38 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/deploy/master/Master.scala | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/0d78b6d9/core/src/main/scala/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala
index 4dd6c44..87a7791 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/spark/deploy/master/Master.scala
@@ -80,6 +80,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
       logInfo("Registering app " + description.name)
       val app = addApplication(description, sender)
       logInfo("Registered app " + description.name + " with ID " + app.id)
+      logInfo("Started App web UI at " + description.appUiUrl)
       waitingApps += app
       context.watch(sender)  // This doesn't work with remote actors but helps for testing
       sender ! RegisteredApplication(app.id)


[24/50] [abbrv] git commit: Merge branch 'master' of github.com:mesos/spark

Posted by ma...@apache.org.
Merge branch 'master' of github.com:mesos/spark


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/238d0e68
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/238d0e68
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/238d0e68

Branch: refs/heads/master
Commit: 238d0e6893b758a541aa782c559215b836ec2db6
Parents: ed7fd50 c7877d5
Author: Matei Zaharia <ma...@gmail.com>
Authored: Mon Jul 15 00:30:32 2013 +0000
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Mon Jul 15 00:30:32 2013 +0000

----------------------------------------------------------------------
 .../spark/deploy/yarn/ApplicationMaster.scala   | 58 ++++++++++++++------
 1 file changed, 40 insertions(+), 18 deletions(-)
----------------------------------------------------------------------



[11/50] [abbrv] git commit: Add `Environment` tab to SparkUI.

Posted by ma...@apache.org.
Add `Environment` tab to SparkUI.

This adds a tab which displays system property and classpath information. This
can be useful in debugging various types of issues such as:

1. Extra/incorrect Hadoop jars being included in the classpath
2. Spark launching with a different JRE version than intended
3. Spark system properties not being set to intended values
4. User added jars that conflict with Spark jars


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/ddb97f0f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/ddb97f0f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/ddb97f0f

Branch: refs/heads/master
Commit: ddb97f0fdf16442afaa9cab656376267e4044510
Parents: 37abe84
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sat Jul 13 16:05:21 2013 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sat Jul 13 16:14:40 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/ui/Page.scala         |  2 +-
 core/src/main/scala/spark/ui/SparkUI.scala      |  7 +--
 core/src/main/scala/spark/ui/UIUtils.scala      |  5 ++
 .../main/scala/spark/ui/env/EnvironmentUI.scala | 58 ++++++++++++++++++++
 4 files changed, 67 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ddb97f0f/core/src/main/scala/spark/ui/Page.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala
index c853b44..ed8f918 100644
--- a/core/src/main/scala/spark/ui/Page.scala
+++ b/core/src/main/scala/spark/ui/Page.scala
@@ -1,3 +1,3 @@
 package spark.ui
 
-private[spark] object Page extends Enumeration { val Storage, Jobs = Value }
\ No newline at end of file
+private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ddb97f0f/core/src/main/scala/spark/ui/SparkUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala
index b3bdc2c..874e5ba 100644
--- a/core/src/main/scala/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/spark/ui/SparkUI.scala
@@ -5,15 +5,13 @@ import javax.servlet.http.HttpServletRequest
 import org.eclipse.jetty.server.{Handler, Server}
 
 import spark.{Logging, SparkContext, Utils}
+import spark.ui.env.EnvironmentUI
 import spark.ui.storage.BlockManagerUI
 import spark.ui.jobs.JobProgressUI
-import spark.ui.UIUtils._
 import spark.ui.JettyUtils._
 
 /** Top level user interface for Spark */
 private[spark] class SparkUI(sc: SparkContext) extends Logging {
-  // TODO(pwendell): It would be nice to add a view that prints out environment information
-
   val host = Utils.localHostName()
   val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt
   var boundPort: Option[Int] = None
@@ -25,7 +23,8 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging {
   )
   val storage = new BlockManagerUI(sc)
   val jobs = new JobProgressUI(sc)
-  val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers
+  val env = new EnvironmentUI(sc)
+  val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ handlers
 
   /** Bind the HTTP server which backs this web interface */
   def bind() {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ddb97f0f/core/src/main/scala/spark/ui/UIUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala
index 7b79290..36d9c47 100644
--- a/core/src/main/scala/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/spark/ui/UIUtils.scala
@@ -19,6 +19,10 @@ private[spark] object UIUtils {
       case Jobs => <li class="active"><a href="/stages">Jobs</a></li>
       case _ => <li><a href="/stages">Jobs</a></li>
     }
+    val environment = page match {
+      case Environment => <li class="active"><a href="/environment">Environment</a></li>
+      case _ => <li><a href="/environment">Environment</a></li>
+    }
 
     <html>
       <head>
@@ -44,6 +48,7 @@ private[spark] object UIUtils {
                     <ul class="nav">
                       {storage}
                       {jobs}
+                      {environment}
                     </ul>
                     <ul id="infolist">
                       <li>Application: <strong>{sc.appName}</strong></li>

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ddb97f0f/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
new file mode 100644
index 0000000..8a3b330
--- /dev/null
+++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
@@ -0,0 +1,58 @@
+package spark.ui.env
+
+import javax.servlet.http.HttpServletRequest
+
+import org.eclipse.jetty.server.Handler
+
+import scala.collection.JavaConversions._
+
+import spark.ui.JettyUtils._
+import spark.ui.UIUtils.headerSparkPage
+import spark.ui.Page.Environment
+import spark.SparkContext
+import spark.ui.UIUtils
+
+import scala.xml.Node
+
+private[spark] class EnvironmentUI(sc: SparkContext) {
+
+  def getHandlers = Seq[(String, Handler)](
+    ("/environment", (request: HttpServletRequest) => envDetails(request))
+  )
+
+  def envDetails(request: HttpServletRequest): Seq[Node] = {
+    val properties = System.getProperties.iterator.toSeq
+
+    val classPathProperty = properties
+        .filter{case (k, v) => k.contains("java.class.path")}
+        .headOption
+        .getOrElse("", "")
+    val sparkProperties = properties.filter(_._1.contains("spark"))
+    val otherProperties = properties.diff(sparkProperties :+ classPathProperty)
+
+    val propertyHeaders = Seq("Name", "Value")
+    def propertyRow(kv: (String, String)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>
+    val propertyTable = UIUtils.listingTable(
+      propertyHeaders, propertyRow, sparkProperties ++ otherProperties)
+
+    val classPathEntries = classPathProperty._2
+        .split(System.getProperty("path.separator", ":"))
+        .filterNot(e => e.isEmpty)
+        .map(e => (e, "System Classpath"))
+    val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")}
+    val addedFiles = sc.addedFiles.iterator.toSeq.map{case (path, time) => (path, "Added By User")}
+    val classPath = addedJars ++ addedFiles ++ classPathEntries
+
+    val classPathHeaders = Seq("Resource", "Source")
+    def classPathRow(data: (String, String)) = <tr><td>{data._1}</td><td>{data._2}</td></tr>
+    val classPathTable = UIUtils.listingTable(classPathHeaders, classPathRow, classPath)
+
+    val content =
+      <span>
+        <h2>System Properties</h2> {propertyTable}
+        <h2>Classpath Entries</h2> {classPathTable}
+      </span>
+
+    headerSparkPage(content, sc, "Environment", Environment)
+  }
+}


[21/50] [abbrv] git commit: Merge pull request #699 from pwendell/ui-env

Posted by ma...@apache.org.
Merge pull request #699 from pwendell/ui-env

Add `Environment` tab to SparkUI.

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/10c05937
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/10c05937
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/10c05937

Branch: refs/heads/master
Commit: 10c05937bdf59abee2b9c6c3ee45ea747c5f6ee4
Parents: 931e4c9 4883586
Author: Matei Zaharia <ma...@gmail.com>
Authored: Sun Jul 14 11:45:18 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Sun Jul 14 11:45:18 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/ui/Page.scala         |  2 +-
 core/src/main/scala/spark/ui/SparkUI.scala      |  7 +-
 core/src/main/scala/spark/ui/UIUtils.scala      |  5 ++
 .../main/scala/spark/ui/env/EnvironmentUI.scala | 69 ++++++++++++++++++++
 4 files changed, 78 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[46/50] [abbrv] Add Apache license headers and LICENSE and NOTICE files

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
index 7e30b8f..c565876 100644
--- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.broadcast
 
 import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/broadcast/MultiTracker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/spark/broadcast/MultiTracker.scala
index 3fd77af..7855d44 100644
--- a/core/src/main/scala/spark/broadcast/MultiTracker.scala
+++ b/core/src/main/scala/spark/broadcast/MultiTracker.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.broadcast
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/broadcast/SourceInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/broadcast/SourceInfo.scala b/core/src/main/scala/spark/broadcast/SourceInfo.scala
index c79bb93..b17ae63 100644
--- a/core/src/main/scala/spark/broadcast/SourceInfo.scala
+++ b/core/src/main/scala/spark/broadcast/SourceInfo.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.broadcast
 
 import java.util.BitSet

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
index c55c476..ea1e9a1 100644
--- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.broadcast
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/ApplicationDescription.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala
index 02193c7..a8b22fb 100644
--- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala
+++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 
 private[spark] class ApplicationDescription(

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/Command.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/Command.scala b/core/src/main/scala/spark/deploy/Command.scala
index 577101e..bad629e 100644
--- a/core/src/main/scala/spark/deploy/Command.scala
+++ b/core/src/main/scala/spark/deploy/Command.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 
 import scala.collection.Map

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/DeployMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala
index 51274ac..e1f8aff 100644
--- a/core/src/main/scala/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/spark/deploy/DeployMessage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 
 import spark.deploy.ExecutorState.ExecutorState

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/ExecutorState.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/spark/deploy/ExecutorState.scala
index 5dc0c54..08c9a3b 100644
--- a/core/src/main/scala/spark/deploy/ExecutorState.scala
+++ b/core/src/main/scala/spark/deploy/ExecutorState.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 
 private[spark] object ExecutorState

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/JsonProtocol.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala
index 335aaeb..64f8962 100644
--- a/core/src/main/scala/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 
 import master.{ApplicationInfo, WorkerInfo}
@@ -62,4 +79,4 @@ private[spark] object JsonProtocol {
     ("executors" -> obj.executors.toList.map(writeExecutorRunner)) ~
     ("finishedexecutors" -> obj.finishedExecutors.toList.map(writeExecutorRunner))
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
index 939f26b..6b8e9f2 100644
--- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 
 import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/WebUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/WebUI.scala b/core/src/main/scala/spark/deploy/WebUI.scala
index 844c414..8ea7792 100644
--- a/core/src/main/scala/spark/deploy/WebUI.scala
+++ b/core/src/main/scala/spark/deploy/WebUI.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy
 
 import java.text.SimpleDateFormat

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/client/Client.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala
index 4af44f9..edefa02 100644
--- a/core/src/main/scala/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/spark/deploy/client/Client.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.client
 
 import spark.deploy._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/client/ClientListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala
index e8c4083..0640244 100644
--- a/core/src/main/scala/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.client
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/client/TestClient.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala
index f195082..4f4daa1 100644
--- a/core/src/main/scala/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/spark/deploy/client/TestClient.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.client
 
 import spark.util.AkkaUtils

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/client/TestExecutor.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/spark/deploy/client/TestExecutor.scala
index 0e46db2..8a22b6b 100644
--- a/core/src/main/scala/spark/deploy/client/TestExecutor.scala
+++ b/core/src/main/scala/spark/deploy/client/TestExecutor.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.client
 
 private[spark] object TestExecutor {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
index 785c16e..15ff919 100644
--- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master
 
 import spark.deploy.ApplicationDescription

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/ApplicationState.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/spark/deploy/master/ApplicationState.scala
index 15016b3..94f0ad8 100644
--- a/core/src/main/scala/spark/deploy/master/ApplicationState.scala
+++ b/core/src/main/scala/spark/deploy/master/ApplicationState.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master
 
 private[spark] object ApplicationState

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
index 48e6055..99b60f7 100644
--- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master
 
 import spark.deploy.ExecutorState

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/Master.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala
index 4dd6c44..e5a7a87 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/spark/deploy/master/Master.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master
 
 import akka.actor._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/MasterArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala
index 3d28eca..d0ec3d5 100644
--- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master
 
 import spark.util.IntParam

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
index 0c08c5f..4135cfe 100644
--- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master
 
 import akka.actor.ActorRef

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/WorkerState.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/WorkerState.scala b/core/src/main/scala/spark/deploy/master/WorkerState.scala
index 0bf3501..3e50b77 100644
--- a/core/src/main/scala/spark/deploy/master/WorkerState.scala
+++ b/core/src/main/scala/spark/deploy/master/WorkerState.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master
 
 private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala
index 8553377..32264af 100644
--- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master.ui
 
 import akka.dispatch.Await

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
index 5e3c5e0..b05197c 100644
--- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master.ui
 
 import akka.dispatch.Await

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala
index 82bc696..04b32c7 100644
--- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.master.ui
 
 import akka.actor.ActorRef
@@ -56,4 +73,4 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten
 private[spark] object MasterWebUI {
   val STATIC_RESOURCE_DIR = "spark/ui/static"
   val DEFAULT_PORT = "8080"
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
index 5d3d54c..8b51ff1 100644
--- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.worker
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/worker/Worker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala
index f20ea42..0bd88ea 100644
--- a/core/src/main/scala/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/spark/deploy/worker/Worker.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.worker
 
 import scala.collection.mutable.{ArrayBuffer, HashMap}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
index 2b96611..9fcd326 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.worker
 
 import spark.util.IntParam

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala
index c515f2e..7548a26 100644
--- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.worker.ui
 
 import akka.dispatch.Await

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala
index ccd55c1..61d4cd6 100644
--- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.deploy.worker.ui
 
 import akka.actor.ActorRef

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/executor/Executor.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala
index 8360547..2e81151 100644
--- a/core/src/main/scala/spark/executor/Executor.scala
+++ b/core/src/main/scala/spark/executor/Executor.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.executor
 
 import java.io.{File, FileOutputStream}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/executor/ExecutorBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/executor/ExecutorBackend.scala b/core/src/main/scala/spark/executor/ExecutorBackend.scala
index e97e509..33a6f8a 100644
--- a/core/src/main/scala/spark/executor/ExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/ExecutorBackend.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.executor
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/executor/ExecutorExitCode.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/spark/executor/ExecutorExitCode.scala
index fd76029..64b9fb8 100644
--- a/core/src/main/scala/spark/executor/ExecutorExitCode.scala
+++ b/core/src/main/scala/spark/executor/ExecutorExitCode.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.executor
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala
index 5beb4d0..09d12fb 100644
--- a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala
+++ b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.executor
 
 import java.net.{URLClassLoader, URL}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
index 10f3531..4961c42 100644
--- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.executor
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
index ebe2ac6..f4003da 100644
--- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.executor
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/executor/TaskMetrics.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala
index 1dc1375..3151627 100644
--- a/core/src/main/scala/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/spark/executor/TaskMetrics.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.executor
 
 class TaskMetrics extends Serializable {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/BufferMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/BufferMessage.scala b/core/src/main/scala/spark/network/BufferMessage.scala
index 7b0e489..e566aea 100644
--- a/core/src/main/scala/spark/network/BufferMessage.scala
+++ b/core/src/main/scala/spark/network/BufferMessage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import java.nio.ByteBuffer
@@ -91,4 +108,4 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId:
       "BufferMessage(id = " + id + ", size = " + size + ")"
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/Connection.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala
index 6e28f67..b66c00b 100644
--- a/core/src/main/scala/spark/network/Connection.scala
+++ b/core/src/main/scala/spark/network/Connection.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/ConnectionManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala
index 624a094..6c4e7dc 100644
--- a/core/src/main/scala/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/spark/network/ConnectionManager.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/ConnectionManagerId.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/ConnectionManagerId.scala b/core/src/main/scala/spark/network/ConnectionManagerId.scala
index b554e84..9d5c518 100644
--- a/core/src/main/scala/spark/network/ConnectionManagerId.scala
+++ b/core/src/main/scala/spark/network/ConnectionManagerId.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import java.net.InetSocketAddress

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/ConnectionManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala
index 533e461..9e3827a 100644
--- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala
+++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/Message.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala
index d4f0361..a25457e 100644
--- a/core/src/main/scala/spark/network/Message.scala
+++ b/core/src/main/scala/spark/network/Message.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/MessageChunk.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/MessageChunk.scala b/core/src/main/scala/spark/network/MessageChunk.scala
index aaf9204..784db5a 100644
--- a/core/src/main/scala/spark/network/MessageChunk.scala
+++ b/core/src/main/scala/spark/network/MessageChunk.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/MessageChunkHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/MessageChunkHeader.scala b/core/src/main/scala/spark/network/MessageChunkHeader.scala
index 3693d50..18d0cbc 100644
--- a/core/src/main/scala/spark/network/MessageChunkHeader.scala
+++ b/core/src/main/scala/spark/network/MessageChunkHeader.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import java.net.InetAddress

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/ReceiverTest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/ReceiverTest.scala b/core/src/main/scala/spark/network/ReceiverTest.scala
index a174d5f..2bbc736 100644
--- a/core/src/main/scala/spark/network/ReceiverTest.scala
+++ b/core/src/main/scala/spark/network/ReceiverTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/SenderTest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/SenderTest.scala b/core/src/main/scala/spark/network/SenderTest.scala
index a4ff69e..542c54c 100644
--- a/core/src/main/scala/spark/network/SenderTest.scala
+++ b/core/src/main/scala/spark/network/SenderTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/netty/FileHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/netty/FileHeader.scala b/core/src/main/scala/spark/network/netty/FileHeader.scala
index aed4254..bf46d32 100644
--- a/core/src/main/scala/spark/network/netty/FileHeader.scala
+++ b/core/src/main/scala/spark/network/netty/FileHeader.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty
 
 import io.netty.buffer._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/netty/ShuffleCopier.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala
index 8d5194a..b01f636 100644
--- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala
+++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty
 
 import java.util.concurrent.Executors

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/network/netty/ShuffleSender.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala
index d6fa4b1..cdf88b0 100644
--- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala
+++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.network.netty
 
 import java.io.File

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/package.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/package.scala b/core/src/main/scala/spark/package.scala
index 389ec4d..b244bfb 100644
--- a/core/src/main/scala/spark/package.scala
+++ b/core/src/main/scala/spark/package.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 /**
  * 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

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/ApproximateActionListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala
index de2dce1..691d939 100644
--- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala
+++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/ApproximateEvaluator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala
index 75713b2..5eae144 100644
--- a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala
+++ b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/BoundedDouble.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/BoundedDouble.scala b/core/src/main/scala/spark/partial/BoundedDouble.scala
index 463c33d..8bdbe6c 100644
--- a/core/src/main/scala/spark/partial/BoundedDouble.scala
+++ b/core/src/main/scala/spark/partial/BoundedDouble.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/CountEvaluator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/CountEvaluator.scala b/core/src/main/scala/spark/partial/CountEvaluator.scala
index daf2c51..6aa9209 100644
--- a/core/src/main/scala/spark/partial/CountEvaluator.scala
+++ b/core/src/main/scala/spark/partial/CountEvaluator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 import cern.jet.stat.Probability

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala
index 01fbb8a..ebe2e5a 100644
--- a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala
+++ b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 import java.util.{HashMap => JHashMap}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala
index c622df5..2dadbbd 100644
--- a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala
+++ b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 import java.util.{HashMap => JHashMap}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
index 20fa55c..ae2b63f 100644
--- a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
+++ b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 import java.util.{HashMap => JHashMap}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/MeanEvaluator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/MeanEvaluator.scala b/core/src/main/scala/spark/partial/MeanEvaluator.scala
index 762c854..5ddcad7 100644
--- a/core/src/main/scala/spark/partial/MeanEvaluator.scala
+++ b/core/src/main/scala/spark/partial/MeanEvaluator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 import cern.jet.stat.Probability

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/PartialResult.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/PartialResult.scala b/core/src/main/scala/spark/partial/PartialResult.scala
index 200ed4e..922a9f9 100644
--- a/core/src/main/scala/spark/partial/PartialResult.scala
+++ b/core/src/main/scala/spark/partial/PartialResult.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 class PartialResult[R](initialVal: R, isFinal: Boolean) {

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/StudentTCacher.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/StudentTCacher.scala b/core/src/main/scala/spark/partial/StudentTCacher.scala
index 443abba..f3bb987 100644
--- a/core/src/main/scala/spark/partial/StudentTCacher.scala
+++ b/core/src/main/scala/spark/partial/StudentTCacher.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 import cern.jet.stat.Probability

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/partial/SumEvaluator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/partial/SumEvaluator.scala b/core/src/main/scala/spark/partial/SumEvaluator.scala
index 58fb60f..4083abe 100644
--- a/core/src/main/scala/spark/partial/SumEvaluator.scala
+++ b/core/src/main/scala/spark/partial/SumEvaluator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.partial
 
 import cern.jet.stat.Probability

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/BlockRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala
index 719d4bf..0ebb722 100644
--- a/core/src/main/scala/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/spark/rdd/BlockRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/CartesianRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala
index 38600b8..150e5bc 100644
--- a/core/src/main/scala/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import java.io.{ObjectOutputStream, IOException}


[36/50] [abbrv] git commit: Merge pull request #705 from rxin/errormessages

Posted by ma...@apache.org.
Merge pull request #705 from rxin/errormessages

Throw a more meaningful message when runJob is called to launch tasks on non-existent partitions.

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/8a8a8f2d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/8a8a8f2d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/8a8a8f2d

Branch: refs/heads/master
Commit: 8a8a8f2de225310410ebe4e5d9c1e9b1ee8ffcf2
Parents: ed8415b 6931660
Author: Matei Zaharia <ma...@gmail.com>
Authored: Mon Jul 15 23:09:21 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Mon Jul 15 23:09:21 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/scheduler/DAGScheduler.scala | 9 +++++++++
 core/src/test/scala/spark/RDDSuite.scala               | 6 ++++++
 2 files changed, 15 insertions(+)
----------------------------------------------------------------------



[06/50] [abbrv] git commit: Merge pull request #691 from karenfeng/logpaging

Posted by ma...@apache.org.
Merge pull request #691 from karenfeng/logpaging

Create log pages

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/5a7835c1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/5a7835c1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/5a7835c1

Branch: refs/heads/master
Commit: 5a7835c152acf0411e117c2fb2bca1177d8938f4
Parents: 71ccca0 73984b9
Author: Matei Zaharia <ma...@gmail.com>
Authored: Fri Jul 12 20:28:21 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Fri Jul 12 20:28:21 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/Utils.scala           |  11 +-
 .../main/scala/spark/deploy/master/Master.scala |   2 +-
 .../deploy/master/ui/ApplicationPage.scala      |   4 +-
 .../main/scala/spark/deploy/worker/Worker.scala |   2 +-
 .../spark/deploy/worker/ui/IndexPage.scala      |  11 +-
 .../spark/deploy/worker/ui/WorkerWebUI.scala    | 103 +++++++++++++++++--
 core/src/test/scala/spark/UtilsSuite.scala      |  49 ++++++++-
 core/src/test/scala/spark/ui/UISuite.scala      |  52 +---------
 8 files changed, 162 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/5a7835c1/core/src/main/scala/spark/Utils.scala
----------------------------------------------------------------------


[04/50] [abbrv] git commit: Merge pull request #692 from Reinvigorate/takeOrdered

Posted by ma...@apache.org.
Merge pull request #692 from Reinvigorate/takeOrdered

adding takeOrdered() to RDD

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/90fc3f30
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/90fc3f30
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/90fc3f30

Branch: refs/heads/master
Commit: 90fc3f30cdc0c08e03ca60703e640d902a26583e
Parents: cd7259b a166232
Author: Matei Zaharia <ma...@gmail.com>
Authored: Fri Jul 12 20:23:36 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Fri Jul 12 20:23:36 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/RDD.scala             | 12 +++++++++-
 .../main/scala/spark/api/java/JavaRDDLike.scala | 25 ++++++++++++++++++++
 core/src/test/scala/spark/RDDSuite.scala        | 20 +++++++++++++++-
 3 files changed, 55 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[23/50] [abbrv] git commit: Make number of blocks in ALS configurable and lower the default

Posted by ma...@apache.org.
Make number of blocks in ALS configurable and lower the default


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/ed7fd501
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/ed7fd501
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/ed7fd501

Branch: refs/heads/master
Commit: ed7fd501cf7ece730cbdee6c152b917cf6bfb16a
Parents: 10c0593
Author: Matei Zaharia <ma...@gmail.com>
Authored: Mon Jul 15 00:30:10 2013 +0000
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Mon Jul 15 00:30:10 2013 +0000

----------------------------------------------------------------------
 mllib/src/main/scala/spark/mllib/recommendation/ALS.scala | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/ed7fd501/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
----------------------------------------------------------------------
diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
index 21eb212..2abaf2f 100644
--- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala
@@ -91,7 +91,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
    */
   def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = {
     val numBlocks = if (this.numBlocks == -1) {
-      math.max(ratings.context.defaultParallelism, ratings.partitions.size)
+      math.max(ratings.context.defaultParallelism, ratings.partitions.size / 2)
     } else {
       this.numBlocks
     }
@@ -384,12 +384,13 @@ object ALS {
   }
 
   def main(args: Array[String]) {
-    if (args.length != 5) {
-      println("Usage: ALS <master> <ratings_file> <rank> <iterations> <output_dir>")
+    if (args.length != 5 && args.length != 6) {
+      println("Usage: ALS <master> <ratings_file> <rank> <iterations> <output_dir> [<blocks>]")
       System.exit(1)
     }
     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.locality.wait", "10000")
     val sc = new SparkContext(master, "ALS")
@@ -397,7 +398,7 @@ object ALS {
       val fields = line.split(',')
       (fields(0).toInt, fields(1).toInt, fields(2).toDouble)
     }
-    val model = ALS.train(ratings, rank, iters)
+    val model = ALS.train(ratings, rank, iters, 0.01, blocks)
     model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") }
                       .saveAsTextFile(outputDir + "/userFeatures")
     model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") }


[31/50] [abbrv] git commit: Merge branch 'ui-802' of https://github.com/karenfeng/spark into ui-802

Posted by ma...@apache.org.
Merge branch 'ui-802' of https://github.com/karenfeng/spark into ui-802


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/eac381a9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/eac381a9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/eac381a9

Branch: refs/heads/master
Commit: eac381a957c1041f4fc65e42342a57a812a277f4
Parents: 3955711 0d78b6d
Author: Karen Feng <ka...@gmail.com>
Authored: Mon Jul 15 15:48:44 2013 -0700
Committer: Karen Feng <ka...@gmail.com>
Committed: Mon Jul 15 15:48:44 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/deploy/master/Master.scala | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------



[45/50] [abbrv] Add Apache license headers and LICENSE and NOTICE files

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/CheckpointRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
index 377b1bd..6794e0e 100644
--- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
index 8966f9f..c540cd3 100644
--- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import java.io.{ObjectOutputStream, IOException}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/CoalescedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
index 6d862c0..2b5bf18 100644
--- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/EmptyRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/spark/rdd/EmptyRDD.scala
index e4dd3a7..d7d4db5 100644
--- a/core/src/main/scala/spark/rdd/EmptyRDD.scala
+++ b/core/src/main/scala/spark/rdd/EmptyRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/FilteredRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala
index c84ec39..783508c 100644
--- a/core/src/main/scala/spark/rdd/FilteredRDD.scala
+++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{OneToOneDependency, RDD, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
index 8ebc778..ed75eac 100644
--- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
+++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{RDD, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/GlommedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala
index e16c7ba..1573f8a 100644
--- a/core/src/main/scala/spark/rdd/GlommedRDD.scala
+++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{RDD, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/HadoopRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala
index 07c1035..d0fdeb7 100644
--- a/core/src/main/scala/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import java.io.EOFException

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/JdbcRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala
index a50f407..5913243 100644
--- a/core/src/main/scala/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import java.sql.{Connection, ResultSet}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
index d283c5b..af8f0a1 100644
--- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{RDD, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
index afb7504..3b4e951 100644
--- a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
+++ b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{RDD, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/MappedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala
index af07311..8b411dd 100644
--- a/core/src/main/scala/spark/rdd/MappedRDD.scala
+++ b/core/src/main/scala/spark/rdd/MappedRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{RDD, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
index 901d01e..17fe805 100644
--- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import java.text.SimpleDateFormat

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
index 07585a8..16ba0c2 100644
--- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
+++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import scala.collection.immutable.NumericRange

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
index 41ff62d..191cfde 100644
--- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
+++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/PipedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala
index c0baf43..2cefdc7 100644
--- a/core/src/main/scala/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/spark/rdd/PipedRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import java.io.PrintWriter

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/SampledRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala
index 243673f..574c9b1 100644
--- a/core/src/main/scala/spark/rdd/SampledRDD.scala
+++ b/core/src/main/scala/spark/rdd/SampledRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import java.util.Random

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/ShuffledRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
index c7d1926..0137f80 100644
--- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/SubtractedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala
index 8a9efc5..0402b9f 100644
--- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import java.util.{HashMap => JHashMap}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/UnionRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala
index 2c52a67..2776826 100644
--- a/core/src/main/scala/spark/rdd/UnionRDD.scala
+++ b/core/src/main/scala/spark/rdd/UnionRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import scala.collection.mutable.ArrayBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala
index b234428..6a4fa13 100644
--- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala
+++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/rdd/ZippedRDD.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala
index f728e93..b1c43b3 100644
--- a/core/src/main/scala/spark/rdd/ZippedRDD.scala
+++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.rdd
 
 import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/ActiveJob.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala
index 105eaec..71cc94e 100644
--- a/core/src/main/scala/spark/scheduler/ActiveJob.scala
+++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import spark.TaskContext

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/DAGScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
index 64ed91f..29e879a 100644
--- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import cluster.TaskInfo

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
index acad915..506c87f 100644
--- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import java.util.Properties

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/InputFormatInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala
index 17d0ea4..65f8c32 100644
--- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala
+++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import spark.Logging

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/JobListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/JobListener.scala b/core/src/main/scala/spark/scheduler/JobListener.scala
index f46b9d5..af108b8 100644
--- a/core/src/main/scala/spark/scheduler/JobListener.scala
+++ b/core/src/main/scala/spark/scheduler/JobListener.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/JobLogger.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala
index 6a9d52f..85b5ddd 100644
--- a/core/src/main/scala/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/spark/scheduler/JobLogger.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import java.io.PrintWriter

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/JobResult.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala
index a0fdf39..a61b335 100644
--- a/core/src/main/scala/spark/scheduler/JobResult.scala
+++ b/core/src/main/scala/spark/scheduler/JobResult.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/JobWaiter.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala
index 6ff2e29..69cd161 100644
--- a/core/src/main/scala/spark/scheduler/JobWaiter.scala
+++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import scala.collection.mutable.ArrayBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/MapStatus.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/spark/scheduler/MapStatus.scala
index 203abb9..2f6a68e 100644
--- a/core/src/main/scala/spark/scheduler/MapStatus.scala
+++ b/core/src/main/scala/spark/scheduler/MapStatus.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import spark.storage.BlockManagerId

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/ResultTask.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala
index 83166bc..361b1e6 100644
--- a/core/src/main/scala/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/spark/scheduler/ResultTask.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import spark._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
index 9564738..1c25605 100644
--- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/SparkListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala
index 8de3aa9..4fb1c5d 100644
--- a/core/src/main/scala/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/spark/scheduler/SparkListener.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import java.util.Properties

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/SplitInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/SplitInfo.scala b/core/src/main/scala/spark/scheduler/SplitInfo.scala
index 6abfb7a..4e3661e 100644
--- a/core/src/main/scala/spark/scheduler/SplitInfo.scala
+++ b/core/src/main/scala/spark/scheduler/SplitInfo.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import collection.mutable.ArrayBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/Stage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala
index 4937eb3..5428dae 100644
--- a/core/src/main/scala/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/spark/scheduler/Stage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import java.net.URI

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/StageInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala
index 8d83ff1..c4026f9 100644
--- a/core/src/main/scala/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/spark/scheduler/StageInfo.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import spark.scheduler.cluster.TaskInfo
@@ -9,4 +26,4 @@ case class StageInfo(
     val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]()
 ) {
   override def toString = stage.rdd.toString
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/Task.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala
index a6462c6..50768d4 100644
--- a/core/src/main/scala/spark/scheduler/Task.scala
+++ b/core/src/main/scala/spark/scheduler/Task.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import spark.serializer.SerializerInstance

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/TaskResult.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala
index 6de0aa7..dc0621e 100644
--- a/core/src/main/scala/spark/scheduler/TaskResult.scala
+++ b/core/src/main/scala/spark/scheduler/TaskResult.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import java.io._

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/TaskScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala
index 7787b54..5188308 100644
--- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
index b75d373..245e7cc 100644
--- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
+++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import spark.scheduler.cluster.TaskInfo

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/TaskSet.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/spark/scheduler/TaskSet.scala
index e4b5fca..dc3550d 100644
--- a/core/src/main/scala/spark/scheduler/TaskSet.scala
+++ b/core/src/main/scala/spark/scheduler/TaskSet.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler
 
 import java.util.Properties

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
index 3a0c29b..7c10074 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import java.lang.{Boolean => JBoolean}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
index 327d679..3d06520 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala
index 8bf8382..8825f2d 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import spark.executor.ExecutorExitCode

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/Pool.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala
index 941ba7a..83708f0 100644
--- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import scala.collection.mutable.ArrayBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala
index 2dd9c05..f557b14 100644
--- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import scala.collection.mutable.ArrayBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala
index 18cc15c..9555402 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import java.io.{File, FileInputStream, FileOutputStream}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
index 8844057..4431744 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import spark.{SparkContext, Utils}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala
index f33310a..69e0ac2 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala
index 6e0c679..4b3e3e5 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 object SchedulingMode extends Enumeration("FAIR","FIFO"){

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 170ede0..55d6c0a 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import spark.{Utils, Logging, SparkContext}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
index 3335294..ac9e5ef 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import spark.TaskState.TaskState

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index 004592a..03a64e0 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
index b41e951..761fdf6 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import java.nio.ByteBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
index 718f26b..a1ebd48 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import spark.Utils

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
index b4dd75d..07c3ddc 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 import scala.collection.mutable.ArrayBuffer

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
index c478243..06d1203 100644
--- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.cluster
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
index b000e32..1f73cb9 100644
--- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.local
 
 import java.io.File

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala
index f12fec4..e662ad6 100644
--- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala
+++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.local
 
 import java.io.File

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
index f4a2994..7bc6040 100644
--- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.mesos
 
 import com.google.protobuf.ByteString

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
index e83368b..75b8268 100644
--- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.scheduler.mesos
 
 import com.google.protobuf.ByteString

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/serializer/Serializer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala
index 2ad73b7..dc94d42 100644
--- a/core/src/main/scala/spark/serializer/Serializer.scala
+++ b/core/src/main/scala/spark/serializer/Serializer.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.serializer
 
 import java.io.{EOFException, InputStream, OutputStream}

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/af3c9d50/core/src/main/scala/spark/serializer/SerializerManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/serializer/SerializerManager.scala b/core/src/main/scala/spark/serializer/SerializerManager.scala
index 60b2aac..b7b2470 100644
--- a/core/src/main/scala/spark/serializer/SerializerManager.scala
+++ b/core/src/main/scala/spark/serializer/SerializerManager.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package spark.serializer
 
 import java.util.concurrent.ConcurrentHashMap


[20/50] [abbrv] git commit: Responding to Matei's review

Posted by ma...@apache.org.
Responding to Matei's review


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/48835868
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/48835868
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/48835868

Branch: refs/heads/master
Commit: 4883586838b960c405af208262aed3ec4b078613
Parents: ddb97f0
Author: Patrick Wendell <pw...@gmail.com>
Authored: Sun Jul 14 10:37:26 2013 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Sun Jul 14 10:37:26 2013 -0700

----------------------------------------------------------------------
 .../main/scala/spark/ui/env/EnvironmentUI.scala | 21 +++++++++++++++-----
 1 file changed, 16 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/48835868/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
index 8a3b330..6b8b9f0 100644
--- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
+++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala
@@ -5,6 +5,7 @@ import javax.servlet.http.HttpServletRequest
 import org.eclipse.jetty.server.Handler
 
 import scala.collection.JavaConversions._
+import scala.util.Properties
 
 import spark.ui.JettyUtils._
 import spark.ui.UIUtils.headerSparkPage
@@ -21,19 +22,27 @@ private[spark] class EnvironmentUI(sc: SparkContext) {
   )
 
   def envDetails(request: HttpServletRequest): Seq[Node] = {
-    val properties = System.getProperties.iterator.toSeq
+    val jvmInformation = Seq(
+      ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)),
+      ("Java Home", Properties.javaHome),
+      ("Scala Version", Properties.versionString),
+      ("Scala Home", Properties.scalaHome)
+    )
+    def jvmRow(kv: (String, String)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>
+    def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation)
 
+    val properties = System.getProperties.iterator.toSeq
     val classPathProperty = properties
         .filter{case (k, v) => k.contains("java.class.path")}
         .headOption
         .getOrElse("", "")
-    val sparkProperties = properties.filter(_._1.contains("spark"))
+    val sparkProperties = properties.filter(_._1.startsWith("spark"))
     val otherProperties = properties.diff(sparkProperties :+ classPathProperty)
 
     val propertyHeaders = Seq("Name", "Value")
     def propertyRow(kv: (String, String)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>
-    val propertyTable = UIUtils.listingTable(
-      propertyHeaders, propertyRow, sparkProperties ++ otherProperties)
+    val sparkPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties)
+    val otherPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties)
 
     val classPathEntries = classPathProperty._2
         .split(System.getProperty("path.separator", ":"))
@@ -49,7 +58,9 @@ private[spark] class EnvironmentUI(sc: SparkContext) {
 
     val content =
       <span>
-        <h2>System Properties</h2> {propertyTable}
+        <h2>Runtime Information</h2> {jvmTable}
+        <h2>Spark Properties</h2> {sparkPropertyTable}
+        <h2>System Properties</h2> {otherPropertyTable}
         <h2>Classpath Entries</h2> {classPathTable}
       </span>
 


[03/50] [abbrv] git commit: Minor style fix

Posted by ma...@apache.org.
Minor style fix


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/08150f19
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/08150f19
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/08150f19

Branch: refs/heads/master
Commit: 08150f19abcf08f2a18305080b08736fb8a33a12
Parents: 6855338
Author: Patrick Wendell <pw...@gmail.com>
Authored: Fri Jul 12 19:32:35 2013 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Fri Jul 12 19:32:35 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/storage/StorageUtils.scala | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/08150f19/core/src/main/scala/spark/storage/StorageUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala
index 3e7fa28..aca16f5 100644
--- a/core/src/main/scala/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/spark/storage/StorageUtils.scala
@@ -47,8 +47,8 @@ object StorageUtils {
 
   /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */
   def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = {
-     val blockLocationPairs = storageStatusList
-       .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort)))
+    val blockLocationPairs = storageStatusList
+      .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort)))
     blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap
   }
 


[34/50] [abbrv] git commit: Merge pull request #703 from karenfeng/ui-802

Posted by ma...@apache.org.
Merge pull request #703 from karenfeng/ui-802

Link to job UI from standalone deploy cluster web UI

Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/ed8415b2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/ed8415b2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/ed8415b2

Branch: refs/heads/master
Commit: ed8415b2fa08c24dab9b435dd388d29416f45527
Parents: e3d3e6f 6dc7c9b
Author: Matei Zaharia <ma...@gmail.com>
Authored: Mon Jul 15 16:41:04 2013 -0700
Committer: Matei Zaharia <ma...@gmail.com>
Committed: Mon Jul 15 16:41:04 2013 -0700

----------------------------------------------------------------------
 core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[08/50] [abbrv] git commit: changing com.google.code.findbugs maven coordinates

Posted by ma...@apache.org.
changing com.google.code.findbugs maven coordinates


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/c4d5b01e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/c4d5b01e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/c4d5b01e

Branch: refs/heads/master
Commit: c4d5b01e44c5f289d67670185a14ff4ccd9537db
Parents: 7dcda9a
Author: seanm <se...@webtrends.com>
Authored: Sat Jul 13 14:56:23 2013 -0600
Committer: seanm <se...@webtrends.com>
Committed: Sat Jul 13 14:56:23 2013 -0600

----------------------------------------------------------------------
 project/SparkBuild.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/c4d5b01e/project/SparkBuild.scala
----------------------------------------------------------------------
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 641d379..0a4c176 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -144,7 +144,7 @@ object SparkBuild extends Build {
 
     libraryDependencies ++= Seq(
       "com.google.guava" % "guava" % "14.0.1",
-      "com.google.code.findbugs" % "jsr305" % "1.3.+",
+      "com.google.code.findbugs" % "jsr305" % "1.3.9",
       "log4j" % "log4j" % "1.2.16",
       "org.slf4j" % "slf4j-api" % slf4jVersion,
       "org.slf4j" % "slf4j-log4j12" % slf4jVersion,