From f678e100536b0cef81e54e30b8bf6438dbf981c1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 21 Nov 2013 07:15:55 +0800 Subject: [PATCH 01/90] Merge branch 'master' of github.com:tbfenet/incubator-spark PartitionPruningRDD is using index from parent I was getting a ArrayIndexOutOfBoundsException exception after doing union on pruned RDD. The index it was using on the partition was the index in the original RDD not the new pruned RDD. (cherry picked from commit 2fead510f74b962b293de4d724136c24a9825271) Signed-off-by: Reynold Xin --- .../spark/rdd/PartitionPruningRDD.scala | 8 +- .../spark/PartitionPruningRDDSuite.scala | 45 ---------- .../spark/rdd/PartitionPruningRDDSuite.scala | 86 +++++++++++++++++++ 3 files changed, 91 insertions(+), 48 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index 165cd412fcfb8..574dd4233fb27 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -33,11 +33,13 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo extends NarrowDependency[T](rdd) { @transient - val partitions: Array[Partition] = rdd.partitions.zipWithIndex - .filter(s => partitionFilterFunc(s._2)) + val partitions: Array[Partition] = rdd.partitions + .filter(s => partitionFilterFunc(s.index)).zipWithIndex .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } - override def getParents(partitionId: Int) = List(partitions(partitionId).index) + override def getParents(partitionId: Int) = { + List(partitions(partitionId).asInstanceOf[PartitionPruningRDDPartition].parentSplit.index) + } } diff --git a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala deleted file mode 100644 index 21f16ef2c6ece..0000000000000 --- a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala +++ /dev/null @@ -1,45 +0,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. - */ - -package org.apache.spark - -import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.{RDD, PartitionPruningRDD} - - -class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { - - test("Pruned Partitions inherit locality prefs correctly") { - class TestPartition(i: Int) extends Partition { - def index = i - } - val rdd = new RDD[Int](sc, Nil) { - override protected def getPartitions = { - Array[Partition]( - new TestPartition(1), - new TestPartition(2), - new TestPartition(3)) - } - def compute(split: Partition, context: TaskContext) = {Iterator()} - } - val prunedRDD = PartitionPruningRDD.create(rdd, {x => if (x==2) true else false}) - val p = prunedRDD.partitions(0) - assert(p.index == 2) - assert(prunedRDD.partitions.length == 1) - } -} diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala new file mode 100644 index 0000000000000..53a7b7c44df1c --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.rdd + +import org.scalatest.FunSuite +import org.apache.spark.{TaskContext, Partition, SharedSparkContext} + + +class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { + + + test("Pruned Partitions inherit locality prefs correctly") { + + val rdd = new RDD[Int](sc, Nil) { + override protected def getPartitions = { + Array[Partition]( + new TestPartition(0, 1), + new TestPartition(1, 1), + new TestPartition(2, 1)) + } + + def compute(split: Partition, context: TaskContext) = { + Iterator() + } + } + val prunedRDD = PartitionPruningRDD.create(rdd, { + x => if (x == 2) true else false + }) + assert(prunedRDD.partitions.length == 1) + val p = prunedRDD.partitions(0) + assert(p.index == 0) + assert(p.asInstanceOf[PartitionPruningRDDPartition].parentSplit.index == 2) + } + + + test("Pruned Partitions can be unioned ") { + + val rdd = new RDD[Int](sc, Nil) { + override protected def getPartitions = { + Array[Partition]( + new TestPartition(0, 4), + new TestPartition(1, 5), + new TestPartition(2, 6)) + } + + def compute(split: Partition, context: TaskContext) = { + List(split.asInstanceOf[TestPartition].testValue).iterator + } + } + val prunedRDD1 = PartitionPruningRDD.create(rdd, { + x => if (x == 0) true else false + }) + + val prunedRDD2 = PartitionPruningRDD.create(rdd, { + x => if (x == 2) true else false + }) + + val merged = prunedRDD1 ++ prunedRDD2 + assert(merged.count() == 2) + val take = merged.take(2) + assert(take.apply(0) == 4) + assert(take.apply(1) == 6) + } +} + +class TestPartition(i: Int, value: Int) extends Partition with Serializable { + def index = i + + def testValue = this.value + +} From d7c6a00cb6f9d80439c998f7b8673fa58848e61e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 22 Nov 2013 10:12:13 +0800 Subject: [PATCH 02/90] Merge pull request #196 from pwendell/master TimeTrackingOutputStream should pass on calls to close() and flush(). Without this fix you get a huge number of open files when running shuffles. (cherry picked from commit f20093c3afa68439b1c9010de189d497df787c2a) Signed-off-by: Reynold Xin --- .../main/scala/org/apache/spark/storage/BlockObjectWriter.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 469e68fed74bb..b4451fc7b8e56 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -93,6 +93,8 @@ class DiskBlockObjectWriter( def write(i: Int): Unit = callWithTiming(out.write(i)) override def write(b: Array[Byte]) = callWithTiming(out.write(b)) override def write(b: Array[Byte], off: Int, len: Int) = callWithTiming(out.write(b, off, len)) + override def close() = out.close() + override def flush() = out.flush() } private val syncWrites = System.getProperty("spark.shuffle.sync", "false").toBoolean From d7ab87e0631bf62ba08263f1f719a3cd28e51f83 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 22 Nov 2013 10:26:39 +0800 Subject: [PATCH 03/90] Merge pull request #193 from aoiwelle/patch-1 Fix Kryo Serializer buffer documentation inconsistency The documentation here is inconsistent with the coded default and other documentation. (cherry picked from commit 086b097e33a2ce622ec6352819bccc92106f43b7) Signed-off-by: Reynold Xin --- docs/tuning.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tuning.md b/docs/tuning.md index f491ae9b95c08..f33fda37ebaba 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -67,7 +67,7 @@ The [Kryo documentation](http://code.google.com/p/kryo/) describes more advanced registration options, such as adding custom serialization code. If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb` -system property. The default is 32, but this value needs to be large enough to hold the *largest* +system property. The default is 2, but this value needs to be large enough to hold the *largest* object you will serialize. Finally, if you don't register your classes, Kryo will still work, but it will have to store the From c59ce18088df21d71006e0216c775f533eb128e2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 24 Nov 2013 11:02:02 +0800 Subject: [PATCH 04/90] Merge pull request #200 from mateiz/hash-fix AppendOnlyMap fixes - Chose a more random reshuffling step for values returned by Object.hashCode to avoid some long chaining that was happening for consecutive integers (e.g. `sc.makeRDD(1 to 100000000, 100).map(t => (t, t)).reduceByKey(_ + _).count`) - Some other small optimizations throughout (see commit comments) (cherry picked from commit 718cc803f7e0600c9ab265022eb6027926a38010) Signed-off-by: Reynold Xin --- .../org/apache/spark/util/AppendOnlyMap.scala | 93 ++++++++++--------- 1 file changed, 50 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala index f60deafc6f323..8bb4ee3bfa22e 100644 --- a/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala @@ -35,6 +35,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi private var capacity = nextPowerOf2(initialCapacity) private var mask = capacity - 1 private var curSize = 0 + private var growThreshold = LOAD_FACTOR * capacity // Holds keys and values in the same array for memory locality; specifically, the order of // elements is key0, value0, key1, value1, key2, value2, etc. @@ -56,7 +57,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi var i = 1 while (true) { val curKey = data(2 * pos) - if (k.eq(curKey) || k == curKey) { + if (k.eq(curKey) || k.equals(curKey)) { return data(2 * pos + 1).asInstanceOf[V] } else if (curKey.eq(null)) { return null.asInstanceOf[V] @@ -80,9 +81,23 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi haveNullValue = true return } - val isNewEntry = putInto(data, k, value.asInstanceOf[AnyRef]) - if (isNewEntry) { - incrementSize() + var pos = rehash(key.hashCode) & mask + var i = 1 + while (true) { + val curKey = data(2 * pos) + if (curKey.eq(null)) { + data(2 * pos) = k + data(2 * pos + 1) = value.asInstanceOf[AnyRef] + incrementSize() // Since we added a new key + return + } else if (k.eq(curKey) || k.equals(curKey)) { + data(2 * pos + 1) = value.asInstanceOf[AnyRef] + return + } else { + val delta = i + pos = (pos + delta) & mask + i += 1 + } } } @@ -104,7 +119,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi var i = 1 while (true) { val curKey = data(2 * pos) - if (k.eq(curKey) || k == curKey) { + if (k.eq(curKey) || k.equals(curKey)) { val newValue = updateFunc(true, data(2 * pos + 1).asInstanceOf[V]) data(2 * pos + 1) = newValue.asInstanceOf[AnyRef] return newValue @@ -161,45 +176,17 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi /** Increase table size by 1, rehashing if necessary */ private def incrementSize() { curSize += 1 - if (curSize > LOAD_FACTOR * capacity) { + if (curSize > growThreshold) { growTable() } } /** - * Re-hash a value to deal better with hash functions that don't differ - * in the lower bits, similar to java.util.HashMap + * Re-hash a value to deal better with hash functions that don't differ in the lower bits. + * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ private def rehash(h: Int): Int = { - val r = h ^ (h >>> 20) ^ (h >>> 12) - r ^ (r >>> 7) ^ (r >>> 4) - } - - /** - * Put an entry into a table represented by data, returning true if - * this increases the size of the table or false otherwise. Assumes - * that "data" has at least one empty slot. - */ - private def putInto(data: Array[AnyRef], key: AnyRef, value: AnyRef): Boolean = { - val mask = (data.length / 2) - 1 - var pos = rehash(key.hashCode) & mask - var i = 1 - while (true) { - val curKey = data(2 * pos) - if (curKey.eq(null)) { - data(2 * pos) = key - data(2 * pos + 1) = value.asInstanceOf[AnyRef] - return true - } else if (curKey.eq(key) || curKey == key) { - data(2 * pos + 1) = value.asInstanceOf[AnyRef] - return false - } else { - val delta = i - pos = (pos + delta) & mask - i += 1 - } - } - return false // Never reached but needed to keep compiler happy + it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) } /** Double the table's size and re-hash everything */ @@ -211,16 +198,36 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] wi throw new Exception("Can't make capacity bigger than 2^29 elements") } val newData = new Array[AnyRef](2 * newCapacity) - var pos = 0 - while (pos < capacity) { - if (!data(2 * pos).eq(null)) { - putInto(newData, data(2 * pos), data(2 * pos + 1)) + val newMask = newCapacity - 1 + // Insert all our old values into the new array. Note that because our old keys are + // unique, there's no need to check for equality here when we insert. + var oldPos = 0 + while (oldPos < capacity) { + if (!data(2 * oldPos).eq(null)) { + val key = data(2 * oldPos) + val value = data(2 * oldPos + 1) + var newPos = rehash(key.hashCode) & newMask + var i = 1 + var keepGoing = true + while (keepGoing) { + val curKey = newData(2 * newPos) + if (curKey.eq(null)) { + newData(2 * newPos) = key + newData(2 * newPos + 1) = value + keepGoing = false + } else { + val delta = i + newPos = (newPos + delta) & newMask + i += 1 + } + } } - pos += 1 + oldPos += 1 } data = newData capacity = newCapacity - mask = newCapacity - 1 + mask = newMask + growThreshold = LOAD_FACTOR * newCapacity } private def nextPowerOf2(n: Int): Int = { From 994956183a6fd565bd585ba6e566b2fbe6cb4d59 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 25 Nov 2013 07:50:46 +0800 Subject: [PATCH 05/90] Merge pull request #197 from aarondav/patrick-fix Fix 'timeWriting' stat for shuffle files Due to concurrent git branches, changes from shuffle file consolidation patch caused the shuffle write timing patch to no longer actually measure the time, since it requires time be measured after the stream has been closed. (cherry picked from commit 972171b9d93b07e8511a2da3a33f897ba033484b) Signed-off-by: Reynold Xin --- .../org/apache/spark/scheduler/ShuffleMapTask.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 1dc71a04282e5..0f2deb4bcbbb2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -167,6 +167,7 @@ private[spark] class ShuffleMapTask( var totalTime = 0L val compressedSizes: Array[Byte] = shuffle.writers.map { writer: BlockObjectWriter => writer.commit() + writer.close() val size = writer.fileSegment().length totalBytes += size totalTime += writer.timeWriting() @@ -184,14 +185,16 @@ private[spark] class ShuffleMapTask( } catch { case e: Exception => // If there is an exception from running the task, revert the partial writes // and throw the exception upstream to Spark. - if (shuffle != null) { - shuffle.writers.foreach(_.revertPartialWrites()) + if (shuffle != null && shuffle.writers != null) { + for (writer <- shuffle.writers) { + writer.revertPartialWrites() + writer.close() + } } throw e } finally { // Release the writers back to the shuffle block manager. if (shuffle != null && shuffle.writers != null) { - shuffle.writers.foreach(_.close()) shuffle.releaseWriters(success) } // Execute the callbacks on task completion. From be9c176a81fba5975c35937ca582a3ef26bda4f5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Nov 2013 18:50:18 -0800 Subject: [PATCH 06/90] Merge pull request #201 from rxin/mappartitions Use the proper partition index in mapPartitionsWIthIndex mapPartitionsWithIndex uses TaskContext.partitionId as the partition index. TaskContext.partitionId used to be identical to the partition index in a RDD. However, pull request #186 introduced a scenario (with partition pruning) that the two can be different. This pull request uses the right partition index in all mapPartitionsWithIndex related calls. Also removed the extra MapPartitionsWIthContextRDD and put all the mapPartitions related functionality in MapPartitionsRDD. (cherry picked from commit 14bb465bb3d65f5b1034ada85cfcad7460034073) Signed-off-by: Reynold Xin --- .../apache/spark/rdd/MapPartitionsRDD.scala | 10 ++--- .../rdd/MapPartitionsWithContextRDD.scala | 41 ------------------- .../main/scala/org/apache/spark/rdd/RDD.scala | 39 ++++++++---------- .../org/apache/spark/CheckpointSuite.scala | 2 - 4 files changed, 22 insertions(+), 70 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 203179c4ea823..ae70d559511c9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -20,18 +20,16 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} -private[spark] -class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( +private[spark] class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( prev: RDD[T], - f: Iterator[T] => Iterator[U], + f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator) preservesPartitioning: Boolean = false) extends RDD[U](prev) { - override val partitioner = - if (preservesPartitioning) firstParent[T].partitioner else None + override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None override def getPartitions: Array[Partition] = firstParent[T].partitions override def compute(split: Partition, context: TaskContext) = - f(firstParent[T].iterator(split, context)) + f(context, split.index, firstParent[T].iterator(split, context)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala deleted file mode 100644 index aea08ff81bfdb..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala +++ /dev/null @@ -1,41 +0,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. - */ - -package org.apache.spark.rdd - -import org.apache.spark.{Partition, TaskContext} - - -/** - * A variant of the MapPartitionsRDD that passes the TaskContext into the closure. From the - * TaskContext, the closure can either get access to the interruptible flag or get the index - * of the partition in the RDD. - */ -private[spark] -class MapPartitionsWithContextRDD[U: ClassManifest, T: ClassManifest]( - prev: RDD[T], - f: (TaskContext, Iterator[T]) => Iterator[U], - preservesPartitioning: Boolean - ) extends RDD[U](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override val partitioner = if (preservesPartitioning) prev.partitioner else None - - override def compute(split: Partition, context: TaskContext) = - f(context, firstParent[T].iterator(split, context)) -} diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6e88be6f6ac64..852c131776b83 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -408,7 +408,6 @@ abstract class RDD[T: ClassManifest]( def pipe(command: String, env: Map[String, String]): RDD[String] = new PipedRDD(this, command, env) - /** * Return an RDD created by piping elements to a forked external process. * The print behavior can be customized by providing two functions. @@ -442,7 +441,8 @@ abstract class RDD[T: ClassManifest]( */ def mapPartitions[U: ClassManifest]( f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { - new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning) + val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(iter) + new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) } /** @@ -451,8 +451,8 @@ abstract class RDD[T: ClassManifest]( */ def mapPartitionsWithIndex[U: ClassManifest]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { - val func = (context: TaskContext, iter: Iterator[T]) => f(context.partitionId, iter) - new MapPartitionsWithContextRDD(this, sc.clean(func), preservesPartitioning) + val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter) + new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) } /** @@ -462,7 +462,8 @@ abstract class RDD[T: ClassManifest]( def mapPartitionsWithContext[U: ClassManifest]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { - new MapPartitionsWithContextRDD(this, sc.clean(f), preservesPartitioning) + val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(context, iter) + new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning) } /** @@ -483,11 +484,10 @@ abstract class RDD[T: ClassManifest]( def mapWith[A: ClassManifest, U: ClassManifest] (constructA: Int => A, preservesPartitioning: Boolean = false) (f: (T, A) => U): RDD[U] = { - def iterF(context: TaskContext, iter: Iterator[T]): Iterator[U] = { - val a = constructA(context.partitionId) + mapPartitionsWithIndex((index, iter) => { + val a = constructA(index) iter.map(t => f(t, a)) - } - new MapPartitionsWithContextRDD(this, sc.clean(iterF _), preservesPartitioning) + }, preservesPartitioning) } /** @@ -498,11 +498,10 @@ abstract class RDD[T: ClassManifest]( def flatMapWith[A: ClassManifest, U: ClassManifest] (constructA: Int => A, preservesPartitioning: Boolean = false) (f: (T, A) => Seq[U]): RDD[U] = { - def iterF(context: TaskContext, iter: Iterator[T]): Iterator[U] = { - val a = constructA(context.partitionId) + mapPartitionsWithIndex((index, iter) => { + val a = constructA(index) iter.flatMap(t => f(t, a)) - } - new MapPartitionsWithContextRDD(this, sc.clean(iterF _), preservesPartitioning) + }, preservesPartitioning) } /** @@ -511,11 +510,10 @@ abstract class RDD[T: ClassManifest]( * partition with the index of that partition. */ def foreachWith[A: ClassManifest](constructA: Int => A)(f: (T, A) => Unit) { - def iterF(context: TaskContext, iter: Iterator[T]): Iterator[T] = { - val a = constructA(context.partitionId) + mapPartitionsWithIndex { (index, iter) => + val a = constructA(index) iter.map(t => {f(t, a); t}) - } - new MapPartitionsWithContextRDD(this, sc.clean(iterF _), true).foreach(_ => {}) + }.foreach(_ => {}) } /** @@ -524,11 +522,10 @@ abstract class RDD[T: ClassManifest]( * partition with the index of that partition. */ def filterWith[A: ClassManifest](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = { - def iterF(context: TaskContext, iter: Iterator[T]): Iterator[T] = { - val a = constructA(context.partitionId) + mapPartitionsWithIndex((index, iter) => { + val a = constructA(index) iter.filter(t => p(t, a)) - } - new MapPartitionsWithContextRDD(this, sc.clean(iterF _), true) + }, preservesPartitioning = true) } /** diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index f26c44d3e76ff..d2226aa5a5663 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -62,8 +62,6 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { testCheckpointing(_.sample(false, 0.5, 0)) testCheckpointing(_.glom()) testCheckpointing(_.mapPartitions(_.map(_.toString))) - testCheckpointing(r => new MapPartitionsWithContextRDD(r, - (context: TaskContext, iter: Iterator[Int]) => iter.map(_.toString), false )) testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString)) testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x)) testCheckpointing(_.pipe(Seq("cat"))) From d21266e9710d7b72218508d050fe6e9fc903944c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 1 Dec 2013 12:46:58 -0800 Subject: [PATCH 07/90] Merge pull request #219 from sundeepn/schedulerexception Scheduler quits when newStage fails The current scheduler thread does not handle exceptions from newStage stage while launching new jobs. The thread fails on any exception that gets triggered at that level, leaving the cluster hanging with no schduler. (cherry picked from commit 740922f25d5f81617fbe02c7bcd1610d6426bbef) Signed-off-by: Reynold Xin --- .../org/apache/spark/scheduler/DAGScheduler.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 0a34a06a7d1eb..4a5426731f266 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -368,7 +368,17 @@ class DAGScheduler( private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { event match { case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => - val finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) + var finalStage: Stage = null + try { + // New stage creation at times and if its not protected, the scheduler thread is killed. + // e.g. it can fail when jobs are run on HadoopRDD whose underlying hdfs files have been deleted + finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) + } catch { + case e: Exception => + logWarning("Creating new stage failed due to exception - job: " + jobId, e) + listener.jobFailed(e) + return false + } val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length + From 8b091febd0a9e63248928888cab83003464a5976 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 19 Nov 2013 16:11:31 -0800 Subject: [PATCH 08/90] Merge pull request #181 from BlackNiuza/fix_tasks_number correct number of tasks in ExecutorsUI Index `a` is not `execId` here (cherry picked from commit f568912f85f58ae152db90f199c1f3a002f270c1) Signed-off-by: Reynold Xin --- .../apache/spark/ui/exec/ExecutorsUI.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 42e9be6e19254..e596690bc3df8 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -76,7 +76,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } - val execInfo = for (b <- 0 until storageStatusList.size) yield getExecInfo(b) + val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execTable = UIUtils.listingTable(execHead, execRow, execInfo) val content = @@ -99,16 +99,17 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { UIUtils.headerSparkPage(content, sc, "Executors (" + execInfo.size + ")", Executors) } - def getExecInfo(a: Int): Seq[String] = { - val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId - val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString - val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString - val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString - val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.get(a.toString).map(l => l.size).getOrElse(0) - val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0) - val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0) + def getExecInfo(statusId: Int): Seq[String] = { + val status = sc.getExecutorStorageStatus(statusId) + val execId = status.blockManagerId.executorId + val hostPort = status.blockManagerId.hostPort + val rddBlocks = status.blocks.size.toString + val memUsed = status.memUsed().toString + val maxMem = status.maxMem.toString + val diskUsed = status.diskUsed().toString + val activeTasks = listener.executorToTasksActive.getOrElse(execId, HashSet.empty[Long]).size + val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) + val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) val totalTasks = activeTasks + failedTasks + completedTasks Seq( From daaaee175a6c07115c8eef85611f3717123c43b3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Dec 2013 14:21:40 -0800 Subject: [PATCH 09/90] Merge pull request #218 from JoshRosen/spark-970-pyspark-unicode-error Fix UnicodeEncodeError in PySpark saveAsTextFile() (SPARK-970) This fixes [SPARK-970](https://spark-project.atlassian.net/browse/SPARK-970), an issue where PySpark's saveAsTextFile() could throw UnicodeEncodeError when called on an RDD of Unicode strings. Please merge this into master and branch-0.8. (cherry picked from commit 8a3475aed66617772f4e98e9f774b109756eb391) Signed-off-by: Reynold Xin --- python/pyspark/rdd.py | 5 ++++- python/pyspark/tests.py | 15 +++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 7019fb8beefc8..0c599e0c5f1a1 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -598,7 +598,10 @@ def saveAsTextFile(self, path): '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n' """ def func(split, iterator): - return (str(x).encode("utf-8") for x in iterator) + for x in iterator: + if not isinstance(x, basestring): + x = unicode(x) + yield x.encode("utf-8") keyed = PipelinedRDD(self, func) keyed._bypass_serializer = True keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 29d6a128f6a9b..d3f6c2bcfbb34 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -19,6 +19,8 @@ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. """ +from fileinput import input +from glob import glob import os import shutil import sys @@ -137,6 +139,19 @@ def func(): self.assertEqual("Hello World from inside a package!", UserClass().hello()) +class TestRDDFunctions(PySparkTestCase): + + def test_save_as_textfile_with_unicode(self): + # Regression test for SPARK-970 + x = u"\u00A1Hola, mundo!" + data = self.sc.parallelize([x]) + tempFile = NamedTemporaryFile(delete=True) + tempFile.close() + data.saveAsTextFile(tempFile.name) + raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) + self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + + class TestIO(PySparkTestCase): def test_stdout_redirection(self): From 31da065b1d08c1fad5283e4bcf8e0ed01818c03e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 21 Oct 2013 20:33:29 -0700 Subject: [PATCH 10/90] Merge pull request #95 from aarondav/perftest Minor: Put StoragePerfTester in org/apache/ (cherry picked from commit a51359c917a9ebe379b32ebc53fd093c454ea195) Signed-off-by: Reynold Xin --- .../scala/{ => org/apache}/spark/storage/StoragePerfTester.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename core/src/main/scala/{ => org/apache}/spark/storage/StoragePerfTester.scala (100%) diff --git a/core/src/main/scala/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala similarity index 100% rename from core/src/main/scala/spark/storage/StoragePerfTester.scala rename to core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala From ba44f21dd7848f1689948d032a34ae92ffadeb31 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 4 Dec 2013 10:28:50 -0800 Subject: [PATCH 11/90] Merge pull request #223 from rxin/transient Mark partitioner, name, and generator field in RDD as @transient. As part of the effort to reduce serialized task size. (cherry picked from commit d6e5473872f405a6f4e466705e33cf893af915c1) Signed-off-by: Patrick Wendell --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 852c131776b83..717247ba2fc4d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -101,7 +101,7 @@ abstract class RDD[T: ClassManifest]( protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ - val partitioner: Option[Partitioner] = None + @transient val partitioner: Option[Partitioner] = None // ======================================================================= // Methods and fields available on all RDDs @@ -114,7 +114,7 @@ abstract class RDD[T: ClassManifest]( val id: Int = sc.newRddId() /** A friendly name for this RDD */ - var name: String = null + @transient var name: String = null /** Assign a name to this RDD */ def setName(_name: String) = { @@ -123,7 +123,7 @@ abstract class RDD[T: ClassManifest]( } /** User-defined generator of this RDD*/ - var generator = Utils.getCallSiteInfo.firstUserClass + @transient var generator = Utils.getCallSiteInfo.firstUserClass /** Reset generator*/ def setGenerator(_generator: String) = { @@ -925,7 +925,7 @@ abstract class RDD[T: ClassManifest]( private var storageLevel: StorageLevel = StorageLevel.NONE /** Record user function generating this RDD. */ - private[spark] val origin = Utils.formatSparkCallSite + @transient private[spark] val origin = Utils.formatSparkCallSite private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T] @@ -940,7 +940,7 @@ abstract class RDD[T: ClassManifest]( def context = sc // Avoid handling doCheckpoint multiple times to prevent excessive recursion - private var doCheckpointCalled = false + @transient private var doCheckpointCalled = false /** * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler From cc33f9fc8a5b33addf23d6ce491aeee2c2ad1b43 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 4 Dec 2013 15:52:07 -0800 Subject: [PATCH 12/90] Merge pull request #227 from pwendell/master Fix small bug in web UI and minor clean-up. There was a bug where sorting order didn't work correctly for write time metrics. I also cleaned up some earlier code that fixed the same issue for read and write bytes. (cherry picked from commit 182f9baeed8e4cc62ca14ae04413394477a7ccfb) Signed-off-by: Patrick Wendell --- .../org/apache/spark/ui/jobs/StagePage.scala | 29 +++++++++---------- 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index c1c7aa70e6c92..c8625716e9d8e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -152,21 +152,18 @@ private[spark] class StagePage(parent: JobProgressUI) { else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L) - var shuffleReadSortable: String = "" - var shuffleReadReadable: String = "" - if (shuffleRead) { - shuffleReadSortable = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => s.remoteBytesRead}.toString() - shuffleReadReadable = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => - Utils.bytesToString(s.remoteBytesRead)}.getOrElse("") - } + val maybeShuffleRead = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => s.remoteBytesRead} + val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") + val shuffleReadReadable = maybeShuffleRead.map{Utils.bytesToString(_)}.getOrElse("") - var shuffleWriteSortable: String = "" - var shuffleWriteReadable: String = "" - if (shuffleWrite) { - shuffleWriteSortable = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleBytesWritten}.toString() - shuffleWriteReadable = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => - Utils.bytesToString(s.shuffleBytesWritten)}.getOrElse("") - } + val maybeShuffleWrite = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleBytesWritten} + val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") + val shuffleWriteReadable = maybeShuffleWrite.map{Utils.bytesToString(_)}.getOrElse("") + + val maybeWriteTime = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleWriteTime} + val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") + val writeTimeReadable = maybeWriteTime.map{ t => t / (1000 * 1000)}.map{ ms => + if (ms == 0) "" else parent.formatDuration(ms)}.getOrElse("") {info.index} @@ -187,8 +184,8 @@ private[spark] class StagePage(parent: JobProgressUI) { }} {if (shuffleWrite) { - {metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => - parent.formatDuration(s.shuffleWriteTime / (1000 * 1000))}.getOrElse("")} + + {writeTimeReadable} {shuffleWriteReadable} From 03edfa5762b918d4a85e0a8de02d644ab959f362 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 4 Dec 2013 16:28:06 -0800 Subject: [PATCH 13/90] Change log for release 0.8.1-incubating --- CHANGES.txt | 1373 +++++++++------------------------------------------ 1 file changed, 245 insertions(+), 1128 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 17f9c093b6288..311d90526e28a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,1253 +1,370 @@ Spark Change Log -Release 0.8.0-incubating +Release 0.8.1-incubating - 2aff798 Sun Sep 15 14:05:04 2013 -0700 - Merge pull request #933 from jey/yarn-typo-fix - [Fix typo in Maven build docs] + cc33f9f Wed Dec 4 15:57:47 2013 -0800 + Merge pull request #227 from pwendell/master - dbd2c4f Sun Sep 15 13:20:41 2013 -0700 - Merge pull request #932 from pwendell/mesos-version - [Bumping Mesos version to 0.13.0] + ba44f21 Wed Dec 4 15:56:58 2013 -0800 + Merge pull request #223 from rxin/transient - 9fb0b9d Sun Sep 15 13:02:53 2013 -0700 - Merge pull request #931 from pwendell/yarn-docs - [Explain yarn.version in Maven build docs] + 31da065 Wed Dec 4 14:01:13 2013 -0800 + Merge pull request #95 from aarondav/perftest - c4c1db2 Fri Sep 13 19:52:12 2013 -0700 - Merge pull request #929 from pwendell/master - [Use different Hadoop version for YARN artifacts.] + daaaee1 Tue Dec 3 14:22:05 2013 -0800 + Merge pull request #218 from JoshRosen/spark-970-pyspark-unicode-error - a310de6 Wed Sep 11 19:36:11 2013 -0700 - Merge pull request #926 from kayousterhout/dynamic - [Changed localProperties to use ThreadLocal (not DynamicVariable).] + 8b091fe Mon Dec 2 21:28:13 2013 -0800 + Merge pull request #181 from BlackNiuza/fix_tasks_number - 58c7d8b Wed Sep 11 17:33:42 2013 -0700 - Merge pull request #927 from benh/mesos-docs - [Updated Spark on Mesos documentation.] + d21266e Sun Dec 1 12:47:30 2013 -0800 + Merge pull request #219 from sundeepn/schedulerexception - 91a59e6 Wed Sep 11 10:21:48 2013 -0700 - Merge pull request #919 from mateiz/jets3t - [Add explicit jets3t dependency, which is excluded in hadoop-client] + be9c176 Tue Nov 26 10:27:41 2013 -0800 + Merge pull request #201 from rxin/mappartitions - b9128d3 Wed Sep 11 10:03:06 2013 -0700 - Merge pull request #922 from pwendell/port-change - [Change default port number from 3030 to 4030.] + 9949561 Mon Nov 25 07:51:23 2013 +0800 + Merge pull request #197 from aarondav/patrick-fix - e07eef8 Wed Sep 11 07:35:39 2013 -0700 - Merge pull request #925 from davidmccauley/master - [SPARK-894 - Not all WebUI fields delivered VIA JSON] + c59ce18 Sun Nov 24 11:04:00 2013 +0800 + Merge pull request #200 from mateiz/hash-fix - 8432f27 Tue Sep 10 23:19:53 2013 -0700 - Merge pull request #923 from haoyuan/master - [fix run-example script] + d7ab87e Fri Nov 22 10:27:16 2013 +0800 + Merge pull request #193 from aoiwelle/patch-1 - d40f140 Tue Sep 10 23:05:29 2013 -0700 - Merge pull request #921 from pwendell/master - [Fix HDFS access bug with assembly build.] + d7c6a00 Fri Nov 22 10:13:37 2013 +0800 + Merge pull request #196 from pwendell/master - 0a6c051 Mon Sep 9 23:37:57 2013 -0700 - Merge pull request #918 from pwendell/branch-0.8 - [Update versions for 0.8.0 release.] + 37126e8 Sun Nov 17 18:53:16 2013 -0800 + Merge pull request #174 from ahirreddy/master - 8c14f4b Mon Sep 9 22:07:58 2013 -0700 - Merge pull request #917 from pwendell/master - [Document libgfortran dependency for MLBase] + 8823057 Sun Nov 17 18:53:04 2013 -0800 + Merge pull request #166 from ahirreddy/simr-spark-ui - c81377b Mon Sep 9 20:16:19 2013 -0700 - Merge pull request #915 from ooyala/master - [Get rid of / improve ugly NPE when Utils.deleteRecursively() fails] + e134ed5 Sun Nov 17 18:51:19 2013 -0800 + Merge pull request #137 from tgravescs/sparkYarnJarsHdfsRebase - 61d2a01 Mon Sep 9 18:21:01 2013 -0700 - Merge pull request #916 from mateiz/mkdist-fix - [Fix copy issue in https://github.com/mesos/spark/pull/899] + af98fbc Sun Nov 17 18:50:23 2013 -0800 + Merge pull request #165 from NathanHowell/kerberos-master - a85758c Mon Sep 9 13:45:40 2013 -0700 - Merge pull request #907 from stephenh/document_coalesce_shuffle - [Add better docs for coalesce.] + a64397b Sun Nov 17 18:49:40 2013 -0800 + Merge pull request #153 from ankurdave/stop-spot-cluster - 084fc36 Mon Sep 9 12:01:35 2013 -0700 - Merge pull request #912 from tgravescs/ganglia-pom - [Add metrics-ganglia to core pom file] + 6c60768 Sun Nov 17 18:46:40 2013 -0800 + Merge pull request #160 from xiajunluan/JIRA-923 - 0456384 Mon Sep 9 09:57:54 2013 -0700 - Merge pull request #911 from pwendell/ganglia-sink - [Adding Manen dependency for Ganglia] + f0d350a Sun Nov 17 18:44:49 2013 -0800 + Merge pull request #175 from kayousterhout/no_retry_not_serializable - bf984e2 Sun Sep 8 23:50:24 2013 -0700 - Merge pull request #890 from mridulm/master - [Fix hash bug] + 9d56371 Sun Nov 17 18:43:59 2013 -0800 + Merge pull request #173 from kayousterhout/scheduler_hang - e9d4f44 Sun Sep 8 23:36:48 2013 -0700 - Merge pull request #909 from mateiz/exec-id-fix - [Fix an instance where full standalone mode executor IDs were passed to] + 24e238b Sun Nov 17 18:42:41 2013 -0800 + Merge pull request #182 from rxin/vector - 2447b1c Sun Sep 8 22:27:49 2013 -0700 - Merge pull request #910 from mateiz/ml-doc-tweaks - [Small tweaks to MLlib docs] + e7927ad Thu Nov 14 10:32:27 2013 -0800 + Merge pull request #169 from kayousterhout/mesos_fix - 7d3204b Sun Sep 8 21:39:12 2013 -0700 - Merge pull request #905 from mateiz/docs2 - [Job scheduling and cluster mode docs] + 41dc566 Thu Nov 14 10:30:57 2013 -0800 + Merge pull request #170 from liancheng/hadooprdd-doc-typo - f1f8371 Sun Sep 8 21:26:11 2013 -0700 - Merge pull request #896 from atalwalkar/master - [updated content] + 333859f Thu Nov 14 10:26:27 2013 -0800 + Merge pull request #171 from RIA-pierre-borckmans/master - f68848d Sun Sep 8 18:32:16 2013 -0700 - Merge pull request #906 from pwendell/ganglia-sink - [Clean-up of Metrics Code/Docs and Add Ganglia Sink] + c856651 Tue Nov 12 10:22:18 2013 -0800 + Merge pull request #164 from tdas/kafka-fix - 0b95799 Sun Sep 8 15:30:16 2013 -0700 - Merge pull request #908 from pwendell/master - [Fix target JVM version in scala build] + 30786c6 Sun Nov 10 11:58:58 2013 -0800 + Merge pull request #157 from rxin/kryo - 04cfb3a Sun Sep 8 10:33:20 2013 -0700 - Merge pull request #898 from ilikerps/660 - [SPARK-660: Add StorageLevel support in Python] + 5ce6c75 Sat Nov 9 22:31:59 2013 -0800 + Merge pull request #147 from JoshRosen/fix-java-api-completeness-checker - 38488ac Sun Sep 8 00:28:53 2013 -0700 - Merge pull request #900 from pwendell/cdh-docs - [Provide docs to describe running on CDH/HDP cluster.] + 1d52b50 Sat Nov 9 15:47:40 2013 -0800 + Merge pull request #149 from tgravescs/fixSecureHdfsAccess - a8e376e Sat Sep 7 21:16:01 2013 -0700 - Merge pull request #904 from pwendell/master - [Adding Apache license to two files] + 32a0c4f Sat Nov 9 15:40:54 2013 -0800 + Merge pull request #155 from rxin/jobgroup - cfde85e Sat Sep 7 13:53:08 2013 -0700 - Merge pull request #901 from ooyala/2013-09/0.8-doc-changes - [0.8 Doc changes for make-distribution.sh] + 07ae524 Sat Nov 9 11:56:46 2013 -0800 + Merge pull request #152 from rxin/repl - 4a7813a Sat Sep 7 13:52:24 2013 -0700 - Merge pull request #903 from rxin/resulttask - [Fixed the bug that ResultTask was not properly deserializing outputId.] + a5916b9 Thu Nov 7 11:08:44 2013 -0800 + Merge pull request #148 from squito/include_appId - afe46ba Sat Sep 7 07:28:51 2013 -0700 - Merge pull request #892 from jey/fix-yarn-assembly - [YARN build fixes] + d5ae953 Wed Nov 6 23:23:12 2013 -0800 + Merge pull request #23 from jerryshao/multi-user - 2eebeff Fri Sep 6 15:25:22 2013 -0700 - Merge pull request #897 from pwendell/master - [Docs describing Spark monitoring and instrumentation] + 1d9412b Wed Nov 6 13:28:02 2013 -0800 + Merge pull request #144 from liancheng/runjob-clean - ddcb9d3 Thu Sep 5 23:54:09 2013 -0700 - Merge pull request #895 from ilikerps/821 - [SPARK-821: Don't cache results when action run locally on driver] + c8e0c0d Wed Nov 6 09:40:00 2013 -0800 + Merge pull request #145 from aarondav/sls-fix - 699c331 Thu Sep 5 20:21:53 2013 -0700 - Merge pull request #891 from xiajunluan/SPARK-864 - [[SPARK-864]DAGScheduler Exception if we delete Worker and StandaloneExecutorBackend then add Worker] + 0f62786 Tue Nov 5 23:14:28 2013 -0800 + Merge pull request #143 from rxin/scheduler-hang - 5c7494d Wed Sep 4 22:47:03 2013 -0700 - Merge pull request #893 from ilikerps/master - [SPARK-884: Add unit test to validate Spark JSON output] + 96670e7 Tue Nov 5 10:43:10 2013 -0800 + Merge pull request #140 from aarondav/merge-75 - a547866 Wed Sep 4 21:11:56 2013 -0700 - Merge pull request #894 from c0s/master - [Updating assembly README to reflect recent changes in the build.] + 0848167 Tue Nov 5 10:42:36 2013 -0800 + Merge pull request #142 from liancheng/dagscheduler-pattern-matching - 19f7027 Tue Sep 3 14:29:10 2013 -0700 - Merge pull request #878 from tgravescs/yarnUILink - [Link the Spark UI up to the Yarn UI ] + 07b3f01 Mon Nov 4 23:32:56 2013 -0800 + Merge pull request #75 from JoshRosen/block-manager-cleanup - 68df246 Tue Sep 3 13:01:17 2013 -0700 - Merge pull request #889 from alig/master - [Return the port the WebUI is bound to (useful if port 0 was used)] + e80d1cf Mon Nov 4 20:47:44 2013 -0800 + Merge pull request #139 from aarondav/shuffle-next - d3dd48f Mon Sep 2 16:44:54 2013 -0700 - Merge pull request #887 from mateiz/misc-fixes - [Miscellaneous fixes for 0.8] + 518cf22 Mon Nov 4 18:21:27 2013 -0800 + Merge pull request #128 from shimingfei/joblogger-doc - 636fc0c Mon Sep 2 11:20:39 2013 -0700 - Merge pull request #886 from mateiz/codec - [Fix spark.io.compression.codec and change default codec to LZF] + 7e00dee Mon Nov 4 17:54:35 2013 -0800 + Merge pull request #130 from aarondav/shuffle - d9a53b9 Sun Sep 1 22:12:30 2013 -0700 - Merge pull request #885 from mateiz/win-py - [Allow PySpark to run on Windows] + 1d11e43 Mon Nov 4 16:30:30 2013 -0800 + Merge pull request #138 from marmbrus/branch-0.8 - 3c520fe Sun Sep 1 17:26:55 2013 -0700 - Merge pull request #884 from mateiz/win-fixes - [Run script fixes for Windows after package & assembly change] + a3544ee Sun Nov 3 23:49:19 2013 -0800 + Merge pull request #70 from rxin/hash1 - f957c26 Sun Sep 1 14:53:57 2013 -0700 - Merge pull request #882 from mateiz/package-rename - [Rename spark package to org.apache.spark] + e094daf Sun Nov 3 23:48:40 2013 -0800 + Merge pull request #129 from velvia/2013-11/document-local-uris - a30fac1 Sun Sep 1 12:27:50 2013 -0700 - Merge pull request #883 from alig/master - [Don't require the spark home environment variable to be set for standalone mode (change needed by SIMR)] + a9e7787 Sun Nov 3 23:48:26 2013 -0800 + Merge pull request #125 from velvia/2013-10/local-jar-uri - 03cc765 Sun Sep 1 10:20:56 2013 -0700 - Merge pull request #881 from pwendell/master - [Extend QuickStart to include next steps] + 57fdb3f Sun Nov 3 23:46:18 2013 -0800 + Merge pull request #117 from stephenh/avoid_concurrent_modification_exception - 0e9565a Sat Aug 31 18:55:41 2013 -0700 - Merge pull request #880 from mateiz/ui-tweaks - [Various UI tweaks] + ec0e4f0 Sun Nov 3 23:45:23 2013 -0800 + Merge pull request #124 from tgravescs/sparkHadoopUtilFix - 2b29a1d Sat Aug 31 17:49:45 2013 -0700 - Merge pull request #877 from mateiz/docs - [Doc improvements for 0.8] + ba0e858 Sun Nov 3 23:43:48 2013 -0800 + Merge pull request #126 from kayousterhout/local_fix - 6edef9c Sat Aug 31 13:39:24 2013 -0700 - Merge pull request #861 from AndreSchumacher/pyspark_sampling_function - [Pyspark sampling function] + 504fe74 Sun Nov 3 23:43:03 2013 -0800 + Merge pull request #118 from JoshRosen/blockinfo-memory-usage - fd89835 Sat Aug 31 13:18:12 2013 -0700 - Merge pull request #870 from JoshRosen/spark-885 - [Don't send SIGINT / ctrl-c to Py4J gateway subprocess] + 25fa229 Sun Nov 3 23:41:19 2013 -0800 + Merge pull request #112 from kayousterhout/ui_task_attempt_id - 618f0ec Fri Aug 30 18:17:13 2013 -0700 - Merge pull request #869 from AndreSchumacher/subtract - [PySpark: implementing subtractByKey(), subtract() and keyBy()] + 57ea854 Sun Nov 3 23:38:37 2013 -0800 + Merge pull request #102 from tdas/transform - 94bb7fd Fri Aug 30 12:05:13 2013 -0700 - Merge pull request #876 from mbautin/master_hadoop_rdd_conf - [Make HadoopRDD's configuration accessible] + 5b45c9b Sat Nov 2 14:42:22 2013 -0700 + Merge pull request #133 from Mistobaan/link_fix - 9e17e45 Fri Aug 30 00:22:53 2013 -0700 - Merge pull request #875 from shivaram/build-fix - [Fix broken build by removing addIntercept] + 054d97b Sat Nov 2 14:37:35 2013 -0700 + Merge pull request #134 from rxin/readme - 016787d Thu Aug 29 22:15:14 2013 -0700 - Merge pull request #863 from shivaram/etrain-ridge - [Adding linear regression and refactoring Ridge regression to use SGD] + 87d4e1c Fri Nov 1 17:58:38 2013 -0700 + Merge pull request #132 from Mistobaan/doc_fix - 852d810 Thu Aug 29 22:13:15 2013 -0700 - Merge pull request #819 from shivaram/sgd-cleanup - [Change SVM to use {0,1} labels] + 3db505c Tue Oct 29 01:42:07 2013 -0400 + Merge pull request #119 from soulmachine/master - ca71620 Thu Aug 29 21:51:14 2013 -0700 - Merge pull request #857 from mateiz/assembly - [Change build and run instructions to use assemblies] + abeca01 Sun Oct 27 22:11:39 2013 -0400 + Merge pull request #115 from aarondav/shuffle-fix - 1528776 Thu Aug 29 21:30:47 2013 -0700 - Merge pull request #874 from jerryshao/fix-report-bug - [Fix removed block zero size log reporting] + 79e5c50 Sat Oct 26 13:05:40 2013 -0700 + Merge pull request #108 from alig/master - abdbacf Wed Aug 28 21:11:31 2013 -0700 - Merge pull request #871 from pwendell/expose-local - [Expose `isLocal` in SparkContext.] + cb24278 Sat Oct 26 11:41:18 2013 -0700 + Merge pull request #113 from pwendell/master - afcade3 Wed Aug 28 20:15:40 2013 -0700 - Merge pull request #873 from pwendell/master - [Hot fix for command runner] + 3ec2f51 Sat Oct 26 11:39:29 2013 -0700 + Merge pull request #111 from kayousterhout/ui_name - baa84e7 Wed Aug 28 12:44:46 2013 -0700 - Merge pull request #865 from tgravescs/fixtmpdir - [Spark on Yarn should use yarn approved directories for spark.local.dir and tmp] + 7d47704 Sat Oct 26 11:36:23 2013 -0700 + Merge pull request #114 from soulmachine/master - cd043cf Tue Aug 27 19:50:32 2013 -0700 - Merge pull request #867 from tgravescs/yarnenvconfigs - [Spark on Yarn allow users to specify environment variables ] + 148509f Thu Oct 24 22:32:23 2013 -0700 + Merge pull request #109 from pwendell/master - 898da7e Mon Aug 26 20:40:49 2013 -0700 - Merge pull request #859 from ianbuss/sbt_opts - [Pass SBT_OPTS environment through to sbt_launcher] + c018c61 Thu Oct 24 17:27:28 2013 -0700 + Merge pull request #97 from ewencp/pyspark-system-properties - 17bafea Mon Aug 26 11:59:32 2013 -0700 - Merge pull request #864 from rxin/json1 - [Revert json library change] + 3500b66 Thu Oct 24 17:27:11 2013 -0700 + Merge pull request #93 from kayousterhout/ui_new_state - f9fc5c1 Sat Aug 24 15:19:56 2013 -0700 - Merge pull request #603 from pwendell/ec2-updates - [Several Improvements to EC2 Scripts] + f5eff85 Thu Oct 24 17:26:44 2013 -0700 + Merge pull request #83 from ewencp/pyspark-accumulator-add-method - d282c1e Fri Aug 23 11:20:20 2013 -0700 - Merge pull request #860 from jey/sbt-ide-fixes - [Fix IDE project generation under SBT] + 59d6f06 Thu Oct 24 17:09:05 2013 -0700 + Merge pull request #106 from pwendell/master - 5a6ac12 Thu Aug 22 22:08:03 2013 -0700 - Merge pull request #701 from ScrapCodes/documentation-suggestions - [Documentation suggestions for spark streaming.] + 76f3c2f Wed Oct 23 18:03:26 2013 -0700 + Merge pull request #103 from JoshRosen/unpersist-fix - 46ea0c1 Thu Aug 22 15:57:28 2013 -0700 - Merge pull request #814 from holdenk/master - [Create less instances of the random class during ALS initialization.] + 534bab2 Tue Oct 22 16:02:45 2013 -0700 + Merge pull request #100 from JoshRosen/spark-902 - 9ac3d62 Thu Aug 22 15:51:10 2013 -0700 - Merge pull request #856 from jey/sbt-fix-hadoop-0.23.9 - [Re-add removed dependency to fix build under Hadoop 0.23.9] + fa9a0e4 Tue Oct 22 12:23:17 2013 -0700 + Merge pull request #90 from pwendell/master - ae8ba83 Thu Aug 22 10:14:54 2013 -0700 - Merge pull request #855 from jey/update-build-docs - [Update build docs] + c449ee1 Tue Oct 22 00:00:35 2013 -0700 + Merge pull request #92 from tgravescs/sparkYarnFixClasspath - 8a36fd0 Thu Aug 22 10:13:35 2013 -0700 - Merge pull request #854 from markhamstra/pomUpdate - [Synced sbt and maven builds to use the same dependencies, etc.] + 498cc6b Mon Oct 21 22:45:31 2013 -0700 + Merge pull request #87 from aarondav/shuffle-base - c2d00f1 Thu Aug 22 10:13:03 2013 -0700 - Merge pull request #832 from alig/coalesce - [Coalesced RDD with locality] + e3ad6a5 Mon Oct 21 20:20:42 2013 -0700 + Revert "Merge pull request #94 from aarondav/mesos-fix" - e6d66c8 Wed Aug 21 17:44:31 2013 -0700 - Merge pull request #853 from AndreSchumacher/double_rdd - [Implementing SPARK-838: Add DoubleRDDFunctions methods to PySpark] + fe974ba Mon Oct 21 20:15:30 2013 -0700 + Merge pull request #94 from aarondav/mesos-fix - 2905611 Tue Aug 20 17:36:14 2013 -0700 - Merge pull request #851 from markhamstra/MutablePairTE - [Removed meaningless types] + 1c3f4bd Mon Oct 21 11:58:22 2013 -0700 + Merge pull request #88 from rxin/clean - d61337f Tue Aug 20 10:06:06 2013 -0700 - Merge pull request #844 from markhamstra/priorityRename - [Renamed 'priority' to 'jobId' and assorted minor changes] + 1a50c79 Sun Oct 20 22:27:29 2013 -0700 + Merge pull request #41 from pwendell/shuffle-benchmark - 8cae72e Mon Aug 19 23:40:04 2013 -0700 - Merge pull request #828 from mateiz/sched-improvements - [Scheduler fixes and improvements] + 37a755c Sun Oct 20 21:04:33 2013 -0700 + Merge pull request #89 from rxin/executor - efeb142 Mon Aug 19 19:23:50 2013 -0700 - Merge pull request #849 from mateiz/web-fixes - [Small fixes to web UI] + ec74428 Sun Oct 20 11:46:14 2013 -0700 + Merge pull request #84 from rxin/kill1 - abdc1f8 Mon Aug 19 18:30:56 2013 -0700 - Merge pull request #847 from rxin/rdd - [Allow subclasses of Product2 in all key-value related classes] + 52d13a6 Sun Oct 20 10:50:54 2013 -0700 + Merge pull request #85 from rxin/clean - 8fa0747 Sun Aug 18 17:02:54 2013 -0700 - Merge pull request #840 from AndreSchumacher/zipegg - [Implementing SPARK-878 for PySpark: adding zip and egg files to context ...] + 919c557 Sat Oct 19 11:29:00 2013 -0700 + Merge pull request #79 from aarondav/scdefaults0.8 - 1e137a5 Sat Aug 17 22:22:32 2013 -0700 - Merge pull request #846 from rxin/rdd - [Two minor RDD refactoring] + f3de2ce Fri Oct 18 23:20:16 2013 -0700 + Merge pull request #76 from pwendell/master - e89ffc7 Fri Aug 16 14:02:34 2013 -0700 - Merge pull request #839 from jegonzal/zip_partitions - [Currying RDD.zipPartitions ] + f181560 Fri Oct 18 23:14:28 2013 -0700 + Merge pull request #68 from mosharaf/master - 1fb1b09 Thu Aug 15 22:15:05 2013 -0700 - Merge pull request #841 from rxin/json - [Use the JSON formatter from Scala library and removed dependency on lift-json.] + eaa2150 Fri Oct 18 23:08:47 2013 -0700 + Merge pull request #74 from rxin/kill - c69c489 Thu Aug 15 20:55:09 2013 -0700 - Merge pull request #843 from Reinvigorate/bug-879 - [fixing typo in conf/slaves] + df21ac8 Thu Oct 17 18:38:46 2013 -0700 + Merge pull request #69 from KarthikTunga/master - 230ab27 Thu Aug 15 17:45:17 2013 -0700 - Merge pull request #834 from Daemoen/master - [Updated json output to allow for display of worker state] + b531552 Thu Oct 17 18:37:22 2013 -0700 + Merge pull request #67 from kayousterhout/remove_tsl - 659553b Thu Aug 15 16:56:31 2013 -0700 - Merge pull request #836 from pwendell/rename - [Rename `memoryBytesToString` and `memoryMegabytesToString`] + 5a73ab7 Thu Oct 17 18:36:36 2013 -0700 + Merge pull request #62 from harveyfeng/master - 28369ff Thu Aug 15 16:44:02 2013 -0700 - Merge pull request #829 from JoshRosen/pyspark-unit-tests-python-2.6 - [Fix PySpark unit tests on Python 2.6] + b6ce111 Thu Oct 17 18:35:33 2013 -0700 + Merge pull request #61 from kayousterhout/daemon_thread - 1a13460 Thu Aug 15 15:50:44 2013 -0700 - Merge pull request #833 from rxin/ui - [Various UI improvements.] + 2760055 Thu Oct 17 18:34:56 2013 -0700 + Merge pull request #59 from rxin/warning - 044a088 Wed Aug 14 20:43:49 2013 -0700 - Merge pull request #831 from rxin/scheduler - [A few small scheduler / job description changes.] + 1e67234 Thu Oct 17 18:33:21 2013 -0700 + Merge pull request #65 from tgravescs/fixYarn - 839f2d4 Wed Aug 14 16:17:23 2013 -0700 - Merge pull request #822 from pwendell/ui-features - [Adding GC Stats to TaskMetrics (and three small fixes)] + d0c9d41 Thu Oct 17 18:32:54 2013 -0700 + Merge pull request #34 from kayousterhout/rename - 63446f9 Wed Aug 14 00:17:07 2013 -0700 - Merge pull request #826 from kayousterhout/ui_fix - [Fixed 2 bugs in executor UI (incl. SPARK-877)] + cee3b43 Wed Oct 16 10:37:55 2013 -0700 + Merge pull request #63 from pwendell/master - 3f14cba Tue Aug 13 20:09:51 2013 -0700 - Merge pull request #825 from shivaram/maven-repl-fix - [Set SPARK_CLASSPATH for maven repl tests] + 1d92983 Tue Oct 15 10:52:15 2013 -0700 + Merge pull request #58 from hsaputra/update-pom-asf - 596adc6 Tue Aug 13 19:41:34 2013 -0700 - Merge pull request #824 from mateiz/mesos-0.12.1 - [Update to Mesos 0.12.1] + c50b016 Mon Oct 14 23:18:37 2013 -0700 + Merge pull request #29 from rxin/kill - d316af9 Tue Aug 13 15:31:01 2013 -0700 - Merge pull request #821 from pwendell/print-launch-command - [Print run command to stderr rather than stdout] + 616ea6f Mon Oct 14 20:59:31 2013 -0700 + Merge pull request #57 from aarondav/bid - 1f79d21 Tue Aug 13 15:23:54 2013 -0700 - Merge pull request #818 from kayousterhout/killed_fix - [Properly account for killed tasks.] + e306de8 Mon Oct 14 20:58:48 2013 -0700 + Merge pull request #53 from witgo/master - 622f83c Tue Aug 13 09:58:52 2013 -0700 - Merge pull request #817 from pwendell/pr_784 - [Minor clean-up in metrics servlet code] + d66c01f Mon Oct 14 20:58:17 2013 -0700 + Merge pull request #19 from aarondav/master-zk - a0133bf Tue Aug 13 09:28:18 2013 -0700 - Merge pull request #784 from jerryshao/dev-metrics-servlet - [Add MetricsServlet for Spark metrics system] + 0fcb234 Mon Oct 14 20:54:34 2013 -0700 + Merge pull request #46 from mateiz/py-sort-update - e2fdac6 Mon Aug 12 21:26:59 2013 -0700 - Merge pull request #802 from stayhf/SPARK-760-Python - [Simple PageRank algorithm implementation in Python for SPARK-760] + f94aa52 Mon Oct 14 20:53:15 2013 -0700 + Merge pull request #44 from mateiz/fast-map - d3525ba Mon Aug 12 21:02:39 2013 -0700 - Merge pull request #813 from AndreSchumacher/add_files_pyspark - [Implementing SPARK-865: Add the equivalent of ADD_JARS to PySpark] + 36f5b08 Mon Oct 14 20:50:28 2013 -0700 + Merge pull request #33 from AndreSchumacher/pyspark_partition_key_change - 9e02da2 Mon Aug 12 20:22:27 2013 -0700 - Merge pull request #812 from shivaram/maven-mllib-tests - [Create SparkContext in beforeAll for MLLib tests] + d615b14 Mon Oct 14 20:48:45 2013 -0700 + Merge pull request #32 from mridulm/master - 65d0d91 Mon Aug 12 19:00:57 2013 -0700 - Merge pull request #807 from JoshRosen/guava-optional - [Change scala.Option to Guava Optional in Java APIs] + 7084217 Mon Oct 14 19:31:22 2013 -0700 + Merge pull request #27 from davidmccauley/master - 4346f0a Mon Aug 12 12:12:12 2013 -0700 - Merge pull request #809 from shivaram/sgd-cleanup - [Clean up scaladoc in ML Lib.] + a35259a Mon Oct 14 19:28:46 2013 -0700 + Merge pull request #26 from Du-Li/master - ea1b4ba Mon Aug 12 08:09:58 2013 -0700 - Merge pull request #806 from apivovarov/yarn-205 - [Changed yarn.version to 2.0.5 in pom.xml] + 62ce4ae Mon Oct 14 19:28:09 2013 -0700 + Merge pull request #25 from CruncherBigData/master - 2a39d2c Sun Aug 11 20:35:09 2013 -0700 - Merge pull request #810 from pwendell/dead_doc_code - [Remove now dead code inside of docs] + 6961744 Mon Oct 14 18:55:41 2013 -0700 + Merge pull request #10 from kayousterhout/results_through-bm - e5b9ed2 Sun Aug 11 17:22:47 2013 -0700 - Merge pull request #808 from pwendell/ui_compressed_bytes - [Report compressed bytes read when calculating TaskMetrics] + ce364c0 Mon Oct 14 18:54:35 2013 -0700 + Merge pull request #4 from MLnick/implicit-als - 3796486 Sun Aug 11 14:51:47 2013 -0700 - Merge pull request #805 from woggle/hadoop-rdd-jobconf - [Use new Configuration() instead of slower new JobConf() in SerializableWritable] + f94bd3f Mon Oct 14 15:10:59 2013 -0700 + Merge pull request #28 from tgravescs/sparYarnAppName - ff9ebfa Sun Aug 11 10:52:55 2013 -0700 - Merge pull request #762 from shivaram/sgd-cleanup - [Refactor SGD options into a new class.] + dc2c90d Mon Oct 14 15:03:34 2013 -0700 + Merge pull request #38 from AndreSchumacher/pyspark_sorting - 95c62ca Sun Aug 11 10:30:52 2013 -0700 - Merge pull request #804 from apivovarov/master - [Fixed path to JavaALS.java and JavaKMeans.java, fixed hadoop2-yarn profi...] + 00a7551 Sat Oct 12 21:26:22 2013 -0700 + Merge pull request #52 from harveyfeng/hadoop-closure - 06e4f2a Sat Aug 10 18:06:23 2013 -0700 - Merge pull request #789 from MLnick/master - [Adding Scala version of PageRank example] + 5383a5a Sat Oct 12 21:26:11 2013 -0700 + Merge pull request #20 from harveyfeng/hadoop-config-cache - 71c63de Sat Aug 10 10:21:20 2013 -0700 - Merge pull request #795 from mridulm/master - [Fix bug reported in PR 791 : a race condition in ConnectionManager and Connection] + 4a2e76a Fri Oct 11 16:09:07 2013 -0700 + Merge pull request #54 from aoiwelle/remove_unused_imports - d17eeb9 Sat Aug 10 09:02:27 2013 -0700 - Merge pull request #785 from anfeng/master - [expose HDFS file system stats via Executor metrics] + 64fae16 Wed Oct 9 21:42:18 2013 -0700 + Merge pull request #47 from xiliu82/branch-0.8 - dce5e47 Fri Aug 9 21:53:45 2013 -0700 - Merge pull request #800 from dlyubimov/HBASE_VERSION - [Pull HBASE_VERSION in the head of sbt build] + dfc62e2 Wed Oct 9 16:55:58 2013 -0700 + Merge pull request #49 from mateiz/kryo-fix-2 - cd247ba Fri Aug 9 20:41:13 2013 -0700 - Merge pull request #786 from shivaram/mllib-java - [Java fixes, tests and examples for ALS, KMeans] + 0b6f047 Wed Oct 9 16:53:31 2013 -0700 + Merge pull request #50 from kayousterhout/SPARK-908 - b09d4b7 Fri Aug 9 13:17:08 2013 -0700 - Merge pull request #799 from woggle/sync-fix - [Remove extra synchronization in ResultTask] + f930dd4 Tue Oct 8 22:58:35 2013 -0700 + Merge pull request #43 from mateiz/kryo-fix - 0bc63bf Fri Aug 9 13:16:25 2013 -0700 - Merge pull request #801 from pwendell/print-launch-command - [Print launch command [Branch 0.8 version]] + 8e9bd93 Mon Oct 7 20:47:09 2013 -0700 + Merge pull request #42 from pwendell/shuffle-read-perf - cc6b92e Fri Aug 9 13:00:33 2013 -0700 - Merge pull request #775 from pwendell/print-launch-command - [Log the launch command for Spark daemons] + f2cdcc4 Mon Oct 7 15:49:32 2013 -0700 + Merge pull request #40 from pwendell/branch-0.8 - f94fc75 Fri Aug 9 10:04:03 2013 -0700 - Merge pull request #788 from shane-huang/sparkjavaopts - [For standalone mode, add worker local env setting of SPARK_JAVA_OPTS as ...] + 023e3fd Mon Oct 7 10:47:45 2013 -0700 + Merge pull request #31 from sundeepn/branch-0.8 - 63b6e02 Thu Aug 8 14:02:02 2013 -0700 - Merge pull request #797 from mateiz/chill-0.3.1 - [Update to Chill 0.3.1] + 3cb9040 Thu Sep 26 14:37:06 2013 -0700 + Merge pull request #17 from rxin/optimize - 9955e5a Thu Aug 8 11:03:38 2013 -0700 - Merge pull request #796 from pwendell/bootstrap-design - [Bootstrap re-design] + 35bcf32 Thu Sep 26 14:17:00 2013 -0700 + Merge pull request #930 from holdenk/master - 5133e4b Wed Aug 7 15:50:45 2013 -0700 - Merge pull request #790 from kayousterhout/fix_throughput - [Fixed issue in UI that decreased scheduler throughput by 5x or more] + 976fe60 Thu Sep 26 14:16:17 2013 -0700 + Merge pull request #14 from kayousterhout/untangle_scheduler - 3c8478e Tue Aug 6 23:25:03 2013 -0700 - Merge pull request #747 from mateiz/improved-lr - [Update the Python logistic regression example] + 8cbc96b Thu Sep 26 13:16:05 2013 -0700 + Merge pull request #7 from wannabeast/memorystore-fixes - 6b043a6 Tue Aug 6 22:31:02 2013 -0700 - Merge pull request #724 from dlyubimov/SPARK-826 - [SPARK-826: fold(), reduce(), collect() always attempt to use java serialization] + 240ca93 Thu Sep 26 13:12:06 2013 -0700 + Merge pull request #9 from rxin/limit - de6c4c9 Tue Aug 6 17:09:50 2013 -0700 - Merge pull request #787 from ash211/master - [Update spark-standalone.md] + a186792 Thu Sep 26 13:10:59 2013 -0700 + Merge pull request #937 from jerryshao/localProperties-fix - df4d10d Tue Aug 6 15:44:05 2013 -0700 - Merge pull request #779 from adatao/adatao-global-SparkEnv - [[HOTFIX] Extend thread safety for SparkEnv.get()] + f3c60c9 Thu Sep 26 13:10:24 2013 -0700 + Merge pull request #941 from ilikerps/master - d2b0f0c Tue Aug 6 14:49:39 2013 -0700 - Merge pull request #770 from stayhf/SPARK-760-Java - [Simple PageRank algorithm implementation in Java for SPARK-760] - - d031f73 Mon Aug 5 22:33:00 2013 -0700 - Merge pull request #782 from WANdisco/master - [SHARK-94 Log the files computed by HadoopRDD and NewHadoopRDD] - - 1b63dea Mon Aug 5 22:21:26 2013 -0700 - Merge pull request #769 from markhamstra/NegativeCores - [SPARK-847 + SPARK-845: Zombie workers and negative cores] - - 828aff7 Mon Aug 5 21:37:33 2013 -0700 - Merge pull request #776 from gingsmith/master - [adding matrix factorization data generator] - - 8b27789 Mon Aug 5 19:14:52 2013 -0700 - Merge pull request #774 from pwendell/job-description - [Show user-defined job name in UI] - - 550b0cf Mon Aug 5 12:10:32 2013 -0700 - Merge pull request #780 from cybermaster/master - [SPARK-850] - - 22abbc1 Fri Aug 2 16:37:59 2013 -0700 - Merge pull request #772 from karenfeng/ui-843 - [Show app duration] - - 9d7dfd2 Thu Aug 1 17:41:58 2013 -0700 - Merge pull request #743 from pwendell/app-metrics - [Add application metrics to standalone master] - - 6d7afd7 Thu Aug 1 17:13:28 2013 -0700 - Merge pull request #768 from pwendell/pr-695 - [Minor clean-up of fair scheduler UI] - - 5e7b38f Thu Aug 1 14:59:33 2013 -0700 - Merge pull request #695 from xiajunluan/pool_ui - [Enhance job ui in spark ui system with adding pool information] - - 0a96493 Thu Aug 1 11:27:17 2013 -0700 - Merge pull request #760 from karenfeng/heading-update - [Clean up web UI page headers] - - cb7dd86 Thu Aug 1 11:06:10 2013 -0700 - Merge pull request #758 from pwendell/master-json - [Add JSON path to master index page] - - 58756b7 Wed Jul 31 23:45:41 2013 -0700 - Merge pull request #761 from mateiz/kmeans-generator - [Add data generator for K-means] - - ecab635 Wed Jul 31 18:16:55 2013 -0700 - Merge pull request #763 from c0s/assembly - [SPARK-842. Maven assembly is including examples libs and dependencies] - - 39c75f3 Wed Jul 31 15:52:36 2013 -0700 - Merge pull request #757 from BlackNiuza/result_task_generation - [Bug fix: SPARK-837] - - b2b86c2 Wed Jul 31 15:51:39 2013 -0700 - Merge pull request #753 from shivaram/glm-refactor - [Build changes for ML lib] - - 14bf2fe Wed Jul 31 14:18:16 2013 -0700 - Merge pull request #749 from benh/spark-executor-uri - [Added property 'spark.executor.uri' for launching on Mesos.] - - 4ba4c3f Wed Jul 31 13:14:49 2013 -0700 - Merge pull request #759 from mateiz/split-fix - [Use the Char version of split() instead of the String one in MLUtils] - - a386ced Wed Jul 31 11:22:50 2013 -0700 - Merge pull request #754 from rxin/compression - [Compression codec change] - - 0be071a Wed Jul 31 11:11:59 2013 -0700 - Merge pull request #756 from cdshines/patch-1 - [Refactored Vector.apply(length, initializer) replacing excessive code with library method] - - d4556f4 Wed Jul 31 08:48:14 2013 -0700 - Merge pull request #751 from cdshines/master - [Cleaned Partitioner & PythonPartitioner source by taking out non-related logic to Utils] - - 29b8cd3 Tue Jul 30 21:30:33 2013 -0700 - Merge pull request #755 from jerryshao/add-apache-header - [Add Apache license header to metrics system] - - e87de03 Tue Jul 30 15:00:08 2013 -0700 - Merge pull request #744 from karenfeng/bootstrap-update - [Use Bootstrap progress bars in web UI] - - ae57020 Tue Jul 30 14:56:41 2013 -0700 - Merge pull request #752 from rxin/master - [Minor mllib cleanup] - - 8aee118 Tue Jul 30 10:27:54 2013 -0700 - Merge pull request #748 from atalwalkar/master - [made SimpleUpdater consistent with other updaters] - - 468a36c Mon Jul 29 19:44:33 2013 -0700 - Merge pull request #746 from rxin/cleanup - [Internal cleanup] - - 1e1ffb1 Mon Jul 29 19:26:19 2013 -0700 - Merge pull request #745 from shivaram/loss-update-fix - [Remove duplicate loss history in Gradient Descent] - - c99b674 Mon Jul 29 16:32:55 2013 -0700 - Merge pull request #735 from karenfeng/ui-807 - [Totals for shuffle data and CPU time] - - fe7298b Mon Jul 29 14:01:00 2013 -0700 - Merge pull request #741 from pwendell/usability - [Fix two small usability issues] - - c34c0f6 Mon Jul 29 13:18:10 2013 -0700 - Merge pull request #731 from pxinghao/master - [Adding SVM and Lasso] - - f3d72ff Fri Jul 26 17:19:27 2013 -0700 - Merge pull request #739 from markhamstra/toolsPom - [Missing tools/pom.xml scalatest dependency] - - cb36677 Fri Jul 26 16:59:30 2013 -0700 - Merge pull request #738 from harsha2010/pruning - [Fix bug in Partition Pruning.] - - f3cf094 Thu Jul 25 14:53:21 2013 -0700 - Merge pull request #734 from woggle/executor-env2 - [Get more env vars from driver rather than worker] - - 51c2427 Thu Jul 25 00:03:11 2013 -0700 - Merge pull request #732 from ryanlecompte/master - [Refactor Kryo serializer support to use chill/chill-java] - - 52723b9 Wed Jul 24 14:33:02 2013 -0700 - Merge pull request #728 from jey/examples-jar-env - [Fix setting of SPARK_EXAMPLES_JAR] - - 20338c2 Wed Jul 24 14:32:24 2013 -0700 - Merge pull request #729 from karenfeng/ui-811 - [Stage Page updates] - - 5584ebc Wed Jul 24 11:46:46 2013 -0700 - Merge pull request #675 from c0s/assembly - [Building spark assembly for further consumption of the Spark project with a deployed cluster] - - a73f3ee Wed Jul 24 08:59:14 2013 -0700 - Merge pull request #671 from jerryshao/master - [Add metrics system for Spark] - - b011329 Tue Jul 23 22:50:09 2013 -0700 - Merge pull request #727 from rxin/scheduler - [Scheduler code style cleanup.] - - 876125b Tue Jul 23 22:28:21 2013 -0700 - Merge pull request #726 from rxin/spark-826 - [SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure] - - 2f1736c Tue Jul 23 15:53:30 2013 -0700 - Merge pull request #725 from karenfeng/task-start - [Creates task start events] - - 5364f64 Tue Jul 23 13:40:34 2013 -0700 - Merge pull request #723 from rxin/mllib - [Made RegressionModel serializable and added unit tests to make sure predict methods would work.] - - f369e0e Tue Jul 23 13:22:27 2013 -0700 - Merge pull request #720 from ooyala/2013-07/persistent-rdds-api - [Add a public method getCachedRdds to SparkContext] - - 401aac8 Mon Jul 22 16:57:16 2013 -0700 - Merge pull request #719 from karenfeng/ui-808 - [Creates Executors tab for Jobs UI] - - 8ae1436 Mon Jul 22 16:03:04 2013 -0700 - Merge pull request #722 from JoshRosen/spark-825 - [Fix bug: DoubleRDDFunctions.sampleStdev() computed non-sample stdev()] - - 15fb394 Sun Jul 21 10:33:38 2013 -0700 - Merge pull request #716 from c0s/webui-port - [Regression: default webui-port can't be set via command line "--webui-port" anymore] - - c40f0f2 Fri Jul 19 13:33:04 2013 -0700 - Merge pull request #711 from shivaram/ml-generators - [Move ML lib data generator files to util/] - - 413b841 Fri Jul 19 13:31:38 2013 -0700 - Merge pull request #717 from viirya/dev1 - [Do not copy local jars given to SparkContext in yarn mode] - - 0d0a47c Thu Jul 18 12:06:37 2013 -0700 - Merge pull request #710 from shivaram/ml-updates - [Updates to LogisticRegression] - - c6235b5 Thu Jul 18 11:43:48 2013 -0700 - Merge pull request #714 from adatao/master - [[BUGFIX] Fix for sbt/sbt script SPARK_HOME setting] - - 009c79e Thu Jul 18 11:41:52 2013 -0700 - Merge pull request #715 from viirya/dev1 - [fix a bug in build process that pulls in two versions of ASM.] - - 985a9e3 Wed Jul 17 22:27:19 2013 -0700 - Merge pull request #712 from stayhf/SPARK-817 - [Consistently invoke bash with /usr/bin/env bash in scripts to make code ...] - - cad48ed Tue Jul 16 21:41:28 2013 -0700 - Merge pull request #708 from ScrapCodes/dependencies-upgrade - [Dependency upgrade Akka 2.0.3 -> 2.0.5] - - 8a8a8f2 Mon Jul 15 23:09:21 2013 -0700 - Merge pull request #705 from rxin/errormessages - [Throw a more meaningful message when runJob is called to launch tasks on non-existent partitions.] - - ed8415b Mon Jul 15 16:41:04 2013 -0700 - Merge pull request #703 from karenfeng/ui-802 - [Link to job UI from standalone deploy cluster web UI] - - e3d3e6f Mon Jul 15 14:59:44 2013 -0700 - Merge pull request #702 from karenfeng/ui-fixes - [Adds app name in HTML page titles on job web UI] - - c7877d5 Sun Jul 14 12:58:13 2013 -0700 - Merge pull request #689 from BlackNiuza/application_status - [Bug fix: SPARK-796] - - 10c0593 Sun Jul 14 11:45:18 2013 -0700 - Merge pull request #699 from pwendell/ui-env - [Add `Environment` tab to SparkUI.] - - 89e8549 Sat Jul 13 16:11:08 2013 -0700 - Merge pull request #698 from Reinvigorate/sm-deps-change - [changing com.google.code.findbugs maven coordinates] - - 77c69ae Fri Jul 12 23:05:21 2013 -0700 - Merge pull request #697 from pwendell/block-locations - [Show block locations in Web UI.] - - 5a7835c Fri Jul 12 20:28:21 2013 -0700 - Merge pull request #691 from karenfeng/logpaging - [Create log pages] - - 71ccca0 Fri Jul 12 20:25:06 2013 -0700 - Merge pull request #696 from woggle/executor-env - [Pass executor env vars (e.g. SPARK_CLASSPATH) to compute-classpath.sh] - - 90fc3f3 Fri Jul 12 20:23:36 2013 -0700 - Merge pull request #692 from Reinvigorate/takeOrdered - [adding takeOrdered() to RDD] - - 018d04c Thu Jul 11 12:48:37 2013 -0700 - Merge pull request #684 from woggle/mesos-classloader - [Explicitly set class loader for MesosSchedulerDriver callbacks.] - - bc19477 Wed Jul 10 22:29:41 2013 -0700 - Merge pull request #693 from c0s/readme - [Updating README to reflect Scala 2.9.3 requirements] - - 7dcda9a Mon Jul 8 23:24:23 2013 -0700 - Merge pull request #688 from markhamstra/scalaDependencies - [Fixed SPARK-795 with explicit dependencies] - - 638927b Mon Jul 8 22:58:50 2013 -0700 - Merge pull request #683 from shivaram/sbt-test-fix - [Remove some stack traces from sbt test output] - - 3c13178 Mon Jul 8 14:50:34 2013 -0700 - Merge pull request #687 from atalwalkar/master - [Added "Labeled" to util functions for labeled data] - - 744da8e Sun Jul 7 17:42:25 2013 -0700 - Merge pull request #679 from ryanlecompte/master - [Make binSearch method tail-recursive for RidgeRegression] - - 3cc6818 Sat Jul 6 19:51:20 2013 -0700 - Merge pull request #668 from shimingfei/guava-14.0.1 - [update guava version from 11.0.1 to 14.0.1] - - 2216188 Sat Jul 6 16:18:15 2013 -0700 - Merge pull request #676 from c0s/asf-avro - [Use standard ASF published avro module instead of a proprietory built one] - - 94871e4 Sat Jul 6 15:26:19 2013 -0700 - Merge pull request #655 from tgravescs/master - [Add support for running Spark on Yarn on a secure Hadoop Cluster] - - 3f918b3 Sat Jul 6 12:45:18 2013 -0700 - Merge pull request #672 from holdenk/master - [s/ActorSystemImpl/ExtendedActorSystem/ as ActorSystemImpl results in a warning] - - 2a36e54 Sat Jul 6 12:43:21 2013 -0700 - Merge pull request #673 from xiajunluan/master - [Add config template file for fair scheduler feature] - - 7ba7fa1 Sat Jul 6 11:45:08 2013 -0700 - Merge pull request #674 from liancheng/master - [Bug fix: SPARK-789] - - f4416a1 Sat Jul 6 11:41:58 2013 -0700 - Merge pull request #681 from BlackNiuza/memory_leak - [Remove active job from idToActiveJob when job finished or aborted] - - e063e29 Fri Jul 5 21:54:52 2013 -0700 - Merge pull request #680 from tdas/master - [Fixed major performance bug in Network Receiver] - - bf1311e Fri Jul 5 17:32:44 2013 -0700 - Merge pull request #678 from mateiz/ml-examples - [Start of ML package] - - 6ad85d0 Thu Jul 4 21:32:29 2013 -0700 - Merge pull request #677 from jerryshao/fix_stage_clean - [Clean StageToInfos periodically when spark.cleaner.ttl is enabled] - - 2e32fc8 Thu Jul 4 12:18:20 2013 -0700 - Merge pull request #666 from c0s/master - [hbase dependency is missed in hadoop2-yarn profile of examples module -] - - 6d60fe5 Mon Jul 1 18:24:03 2013 -0700 - Merge pull request #666 from c0s/master - [hbase dependency is missed in hadoop2-yarn profile of examples module] - - ccfe953 Sat Jun 29 17:57:53 2013 -0700 - Merge pull request #577 from skumargithub/master - [Example of cumulative counting using updateStateByKey] - - 50ca176 Thu Jun 27 22:24:52 2013 -0700 - Merge pull request #664 from pwendell/test-fix - [Removing incorrect test statement] - - e49bc8c Wed Jun 26 11:13:33 2013 -0700 - Merge pull request #663 from stephenh/option_and_getenv - [Be cute with Option and getenv.] - - f5e32ed Tue Jun 25 09:16:57 2013 -0700 - Merge pull request #661 from mesos/streaming - [Kafka fixes and DStream.count fix for master] - - 1249e91 Mon Jun 24 21:46:33 2013 -0700 - Merge pull request #572 from Reinvigorate/sm-block-interval - [Adding spark.streaming.blockInterval property] - - cfcda95 Mon Jun 24 21:44:50 2013 -0700 - Merge pull request #571 from Reinvigorate/sm-kafka-serializers - [Surfacing decoders on KafkaInputDStream] - - 575aff6 Mon Jun 24 21:35:50 2013 -0700 - Merge pull request #567 from Reinvigorate/sm-count-fix - [Fixing count() in Spark Streaming] - - 3e61bef Sat Jun 22 16:22:47 2013 -0700 - Merge pull request #648 from shivaram/netty-dbg - [Shuffle fixes and cleanup] - - 1ef5d0d Sat Jun 22 09:35:57 2013 -0700 - Merge pull request #644 from shimingfei/joblogger - [add Joblogger to Spark (on new Spark code)] - - 7e4b266 Sat Jun 22 07:53:18 2013 -0700 - Merge pull request #563 from jey/python-optimization - [Optimize PySpark worker invocation] - - 71030ba Wed Jun 19 15:21:03 2013 -0700 - Merge pull request #654 from lyogavin/enhance_pipe - [fix typo and coding style in #638] - - 73f4c7d Tue Jun 18 04:21:17 2013 -0700 - Merge pull request #605 from esjewett/SPARK-699 - [Add hBase example (retry of pull request #596)] - - 9933836 Tue Jun 18 02:41:10 2013 -0700 - Merge pull request #647 from jerryshao/master - [Reduce ZippedPartitionsRDD's getPreferredLocations complexity from O(2^2n) to O(2^n)] - - db42451 Mon Jun 17 15:26:36 2013 -0700 - Merge pull request #643 from adatao/master - [Bug fix: Zero-length partitions result in NaN for overall mean & variance] - - e82a2ff Mon Jun 17 15:13:15 2013 -0700 - Merge pull request #653 from rxin/logging - [SPARK-781: Log the temp directory path when Spark says "Failed to create temp directory."] - - e6d1277 Mon Jun 17 12:56:25 2013 -0700 - Merge pull request #638 from lyogavin/enhance_pipe - [Enhance pipe to support more features we can do in hadoop streaming] - - f961aac Sat Jun 15 00:53:41 2013 -0700 - Merge pull request #649 from ryanlecompte/master - [Add top K method to RDD using a bounded priority queue] - - 6602d94 Fri Jun 14 10:41:31 2013 -0700 - Merge pull request #651 from rxin/groupbykey - [SPARK-772 / SPARK-774: groupByKey and cogroup should disable map side combine] - - d93851a Thu Jun 13 13:38:45 2013 -0700 - Merge pull request #645 from pwendell/compression - [Adding compression to Hadoop save functions] - - f1da591 Wed Jun 12 17:55:08 2013 -0700 - Merge pull request #646 from markhamstra/jvmArgs - [Fixed jvmArgs in maven build.] - - 0e94b73 Mon Jun 10 13:00:31 2013 -0700 - Merge pull request #625 from stephenh/fix-start-slave - [Fix start-slave not passing instance number to spark-daemon.] - - 74b91d5 Sat Jun 8 01:19:40 2013 -0700 - Merge pull request #629 from c0s/master - [Sometime Maven build runs out of PermGen space.] - - c8fc423 Fri Jun 7 22:43:18 2013 -0700 - Merge pull request #631 from jerryshao/master - [Fix block manager UI display issue when enable spark.cleaner.ttl] - - 1ae60bc Fri Jun 7 22:39:06 2013 -0700 - Merge pull request #634 from xiajunluan/master - [[Spark-753] Fix ClusterSchedulSuite unit test failed ] - - fff3728 Tue Jun 4 16:09:50 2013 -0700 - Merge pull request #640 from pwendell/timeout-update - [Fixing bug in BlockManager timeout] - - f420d4f Tue Jun 4 15:25:58 2013 -0700 - Merge pull request #639 from pwendell/timeout-update - [Bump akka and blockmanager timeouts to 60 seconds] - - 84530ba Fri May 31 17:06:13 2013 -0700 - Merge pull request #636 from rxin/unpersist - [Unpersist More block manager cleanup.] - - ef77bb7 Thu May 30 14:50:06 2013 -0700 - Merge pull request #627 from shivaram/master - [Netty and shuffle bug fixes] - - 8cb8178 Thu May 30 14:17:44 2013 -0700 - Merge pull request #628 from shivaram/zero-block-size - [Skip fetching zero-sized blocks in NIO.] - - 6ed7139 Wed May 29 10:14:22 2013 -0700 - Merge pull request #626 from stephenh/remove-add-if-no-port - [Remove unused addIfNoPort.] - - 41d230c Tue May 28 23:35:24 2013 -0700 - Merge pull request #611 from squito/classloader - [Use default classloaders for akka & deserializing task results] - - 3db1e17 Mon May 27 21:31:43 2013 -0700 - Merge pull request #620 from jerryshao/master - [Fix CheckpointRDD java.io.FileNotFoundException when calling getPreferredLocations] - - 3d4891d Sat May 25 23:38:05 2013 -0700 - Merge pull request #621 from JoshRosen/spark-613 - [Use ec2-metadata in start-slave.sh to detect if running on EC2] - - e8d4b6c Sat May 25 21:09:03 2013 -0700 - Merge pull request #529 from xiajunluan/master - [[SPARK-663]Implement Fair Scheduler in Spark Cluster Scheduler ] - - 9a3c344 Sat May 25 17:53:43 2013 -0700 - Merge pull request #624 from rxin/master - [NonJavaSerializableClass should not be Java serializable...] - - 24e41aa Fri May 24 16:48:52 2013 -0700 - Merge pull request #623 from rxin/master - [Automatically configure Netty port.] - - 69161f9 Fri May 24 14:42:13 2013 -0700 - Merge pull request #622 from rxin/master - [bug fix: Shuffle block iterator is ignoring the shuffle serializer setting.] - - dbbedfc Thu May 23 23:11:06 2013 -0700 - Merge pull request #616 from jey/maven-netty-exclusion - [Exclude old versions of Netty from Maven-based build] - - a2b0a79 Tue May 21 18:16:20 2013 -0700 - Merge pull request #619 from woggling/adjust-sampling - [Use ARRAY_SAMPLE_SIZE constant instead of hard-coded 100.0 in SizeEstimator] - - 66dac44 Tue May 21 11:41:42 2013 -0700 - Merge pull request #618 from woggling/dead-code-disttest - [DistributedSuite: remove dead code] - - 5912cc4 Fri May 17 19:58:40 2013 -0700 - Merge pull request #610 from JoshRosen/spark-747 - [Throw exception if TaskResult exceeds Akka frame size] - - 6c27c38 Thu May 16 17:33:56 2013 -0700 - Merge pull request #615 from rxin/build-fix - [Maven build fix & two other small changes] - - 2f576ab Wed May 15 18:06:24 2013 -0700 - Merge pull request #602 from rxin/shufflemerge - [Manual merge & cleanup of Shane's Shuffle Performance Optimization] - - 48c6f46 Wed May 15 10:47:19 2013 -0700 - Merge pull request #612 from ash211/patch-4 - [Docs: Mention spark shell's default for MASTER] - - 203d7b7 Wed May 15 00:47:20 2013 -0700 - Merge pull request #593 from squito/driver_ui_link - [Master UI has link to Application UI] - - 016ac86 Mon May 13 21:45:36 2013 -0700 - Merge pull request #601 from rxin/emptyrdd-master - [EmptyRDD (master branch 0.8)] - - 4b354e0 Mon May 13 17:39:19 2013 -0700 - Merge pull request #589 from mridulm/master - [Add support for instance local scheduling] - - 5dbc9b2 Sun May 12 11:03:10 2013 -0700 - Merge pull request #608 from pwendell/SPARK-738 - [SPARK-738: Spark should detect and wrap nonserializable exceptions] - - 63e1999 Fri May 10 13:54:03 2013 -0700 - Merge pull request #606 from markhamstra/foreachPartition_fix - [Actually use the cleaned closure in foreachPartition] - - 42bbe89 Wed May 8 22:30:31 2013 -0700 - Merge pull request #599 from JoshRosen/spark-670 - [Fix SPARK-670: EC2 'start' command should require -i option.] - - 0f1b7a0 Wed May 8 13:38:50 2013 -0700 - Merge pull request #596 from esjewett/master - [hBase example] - - 7af92f2 Sat May 4 22:29:17 2013 -0700 - Merge pull request #597 from JoshRosen/webui-fixes - [Two minor bug fixes for Spark Web UI] - - c74ce60 Sat May 4 22:26:35 2013 -0700 - Merge pull request #598 from rxin/blockmanager - [Fixed flaky unpersist test in DistributedSuite.] - - 3bf2c86 Fri May 3 18:27:30 2013 -0700 - Merge pull request #594 from shivaram/master - [Add zip partitions to Java API] - - 2484ad7 Fri May 3 17:08:55 2013 -0700 - Merge pull request #587 from rxin/blockmanager - [A set of shuffle map output related changes] - - 6fe9d4e Thu May 2 21:33:56 2013 -0700 - Merge pull request #592 from woggling/localdir-fix - [Don't accept generated local directory names that can't be created] - - 538ee75 Thu May 2 09:01:42 2013 -0700 - Merge pull request #581 from jerryshao/master - [fix [SPARK-740] block manage UI throws exception when enabling Spark Streaming] - - 9abcbcc Wed May 1 22:45:10 2013 -0700 - Merge pull request #591 from rxin/removerdd - [RDD.unpersist: probably the most desired feature of Spark] - - aa8fe1a Tue Apr 30 22:30:18 2013 -0700 - Merge pull request #586 from mridulm/master - [Pull request to address issues Reynold Xin reported] - - f708dda Tue Apr 30 07:51:40 2013 -0700 - Merge pull request #585 from pwendell/listener-perf - [[Fix SPARK-742] Task Metrics should not employ per-record timing by default] - - 68c07ea Sun Apr 28 20:19:33 2013 -0700 - Merge pull request #582 from shivaram/master - [Add zip partitions interface] - - f6ee9a8 Sun Apr 28 15:36:04 2013 -0700 - Merge pull request #583 from mridulm/master - [Fix issues with streaming test cases after yarn branch merge] - - cf54b82 Thu Apr 25 11:45:58 2013 -0700 - Merge pull request #580 from pwendell/quickstart - [SPARK-739 Have quickstart standlone job use README] - - 118a6c7 Wed Apr 24 08:42:30 2013 -0700 - Merge pull request #575 from mridulm/master - [Manual merge of yarn branch to trunk] - - 5d8a71c Tue Apr 16 19:48:02 2013 -0700 - Merge pull request #570 from jey/increase-codecache-size - [Increase ReservedCodeCacheSize for sbt] - - ec5e553 Sun Apr 14 08:20:13 2013 -0700 - Merge pull request #558 from ash211/patch-jackson-conflict - [Don't pull in old versions of Jackson via hadoop-core] - - c1c219e Sun Apr 14 08:11:23 2013 -0700 - Merge pull request #564 from maspotts/master - [Allow latest scala in PATH, with SCALA_HOME as override (instead of vice-versa)] - - 7c10b3e Fri Apr 12 20:55:22 2013 -0700 - Merge pull request #565 from andyk/master - [Update wording of section on RDD operations in quick start guide in docs] - - 077ae0a Thu Apr 11 19:34:14 2013 -0700 - Merge pull request #561 from ash211/patch-4 - [Add details when BlockManager heartbeats time out] - - c91ff8d Wed Apr 10 15:08:23 2013 -0700 - Merge pull request #560 from ash211/patch-3 - [Typos: cluser -> cluster] - - 7cd83bf Tue Apr 9 22:07:35 2013 -0700 - Merge pull request #559 from ash211/patch-example-whitespace - [Uniform whitespace across scala examples] - - 271a4f3 Tue Apr 9 22:04:52 2013 -0700 - Merge pull request #555 from holdenk/master - [Retry failed ssh commands in the ec2 python script.] - - 8ac9efb Tue Apr 9 13:50:50 2013 -0700 - Merge pull request #527 from Reinvigorate/sm-kafka-cleanup - [KafkaInputDStream fixes and improvements] - - eed54a2 Mon Apr 8 09:44:30 2013 -0700 - Merge pull request #553 from pwendell/akka-standalone - [SPARK-724 - Have Akka logging enabled by default for standalone daemons] - - b362df3 Sun Apr 7 17:17:52 2013 -0700 - Merge pull request #552 from MLnick/master - [Bumping version for Twitter Algebird to latest] - - 4b30190 Sun Apr 7 17:15:10 2013 -0700 - Merge pull request #554 from andyk/scala2.9.3 - [Fixes SPARK-723 - Update build to Scala 2.9.3] - - dfe98ca Tue Apr 2 19:24:12 2013 -0700 - Merge pull request #550 from erikvanoosten/master - [corrected Algebird example] - - b5d7830 Tue Apr 2 19:23:45 2013 -0700 - Merge pull request #551 from jey/python-bugfixes - [Python bugfixes] - - 2be2295 Sun Mar 31 18:09:14 2013 -0700 - Merge pull request #548 from markhamstra/getWritableClass_filter - [Fixed broken filter in getWritableClass[T]] - - 9831bc1 Fri Mar 29 22:16:22 2013 -0700 - Merge pull request #539 from cgrothaus/fix-webui-workdirpath - [Bugfix: WorkerWebUI must respect workDirPath from Worker] - - 3cc8ab6 Fri Mar 29 22:14:07 2013 -0700 - Merge pull request #541 from stephenh/shufflecoalesce - [Add a shuffle parameter to coalesce.] - - cad507a Fri Mar 29 22:13:12 2013 -0700 - Merge pull request #547 from jey/maven-streaming-tests-initialization-fix - [Move streaming test initialization into 'before' blocks] - - a98996d Fri Mar 29 22:12:15 2013 -0700 - Merge pull request #545 from ash211/patch-1 - [Don't use deprecated Application in example] - - 104c694 Fri Mar 29 22:11:50 2013 -0700 - Merge pull request #546 from ash211/patch-2 - [Update tuning.md] - - bc36ee4 Tue Mar 26 15:05:13 2013 -0700 - Merge pull request #543 from holdenk/master - [Re-enable deprecation warnings and fix deprecated warning.] - - b8949ca Sat Mar 23 07:19:34 2013 -0700 - Merge pull request #505 from stephenh/volatile - [Make Executor fields volatile since they're read from the thread pool.] - - fd53f2f Sat Mar 23 07:13:21 2013 -0700 - Merge pull request #510 from markhamstra/WithThing - [mapWith, flatMapWith and filterWith] - - 4c5efcf Wed Mar 20 19:29:23 2013 -0700 - Merge pull request #532 from andyk/master - [SPARK-715: Adds instructions for building with Maven to documentation] - - 3558849 Wed Mar 20 19:27:47 2013 -0700 - Merge pull request #538 from rxin/cogroup - [Added mapSideCombine flag to CoGroupedRDD. Added unit test for CoGroupedRDD.] - - ca4d083 Wed Mar 20 11:22:36 2013 -0700 - Merge pull request #528 from MLnick/java-examples - [[SPARK-707] Adding Java versions of Pi, LogQuery and K-Means examples] - - b812e6b Wed Mar 20 11:21:02 2013 -0700 - Merge pull request #526 from markhamstra/foldByKey - [Add foldByKey] - - 945d1e7 Tue Mar 19 21:59:06 2013 -0700 - Merge pull request #536 from sasurfer/master - [CoalescedRDD for many partitions] - - 1cbbe94 Tue Mar 19 21:34:34 2013 -0700 - Merge pull request #534 from stephenh/removetrycatch - [Remove try/catch block that can't be hit.] - - 71e53f8 Tue Mar 19 21:31:41 2013 -0700 - Merge pull request #537 from wishbear/configurableInputFormat - [call setConf from input format if it is Configurable] - - c1e9cdc Sat Mar 16 11:47:45 2013 -0700 - Merge pull request #525 from stephenh/subtractByKey - [Add PairRDDFunctions.subtractByKey.] - - cdbfd1e Fri Mar 15 15:13:28 2013 -0700 - Merge pull request #516 from squito/fix_local_metrics - [Fix local metrics] - - f9fa2ad Fri Mar 15 15:12:43 2013 -0700 - Merge pull request #530 from mbautin/master-update-log4j-and-make-compile-in-IntelliJ - [Add a log4j compile dependency to fix build in IntelliJ] - - 4032beb Wed Mar 13 19:29:46 2013 -0700 - Merge pull request #521 from stephenh/earlyclose - [Close the reader in HadoopRDD as soon as iteration end.] - - 3c97276 Wed Mar 13 19:25:08 2013 -0700 - Merge pull request #524 from andyk/master - [Fix broken link to YARN documentation] - - 1c3d981 Wed Mar 13 19:23:48 2013 -0700 - Merge pull request #517 from Reinvigorate/sm-build-fixes - [Build fixes for streaming /w SBT] - - 2d477fd Wed Mar 13 06:49:16 2013 -0700 - Merge pull request #523 from andyk/master - [Fix broken link in Quick Start] - - 00c4d23 Tue Mar 12 22:19:00 2013 -0700 - Merge pull request #518 from woggling/long-bm-sizes - [Send block sizes as longs in BlockManager updates] - - cbf8f0d Mon Mar 11 00:23:57 2013 -0700 - Merge pull request #513 from MLnick/bagel-caching - [Adds choice of persistence level to Bagel.] - - 91a9d09 Sun Mar 10 15:48:23 2013 -0700 - Merge pull request #512 from patelh/fix-kryo-serializer - [Fix reference bug in Kryo serializer, add test, update version] - - 557cfd0 Sun Mar 10 15:44:57 2013 -0700 - Merge pull request #515 from woggling/deploy-app-death - [Notify standalone deploy client of application death.] - - 04fb81f Sun Mar 3 17:20:07 2013 -0800 - Merge pull request #506 from rxin/spark-706 - [Fixed SPARK-706: Failures in block manager put leads to read task hanging.] - - 6cf4be4 Sun Mar 3 17:16:22 2013 -0800 - Merge pull request #462 from squito/stageInfo - [Track assorted metrics for each task, report summaries to user at stage completion] - - 6bfc7ca Sat Mar 2 22:14:49 2013 -0800 - Merge pull request #504 from mosharaf/master - [Worker address was getting removed when removing an app.] - - 94b3db1 Sat Mar 2 22:13:52 2013 -0800 - Merge pull request #508 from markhamstra/TestServerInUse - [Avoid bind failure in InputStreamsSuite] - - 25c71d3 Fri Mar 1 08:00:18 2013 -0800 - Merge pull request #507 from markhamstra/poms271 - [bump version to 0.7.1-SNAPSHOT in the subproject poms] + d5a8dbf Thu Sep 26 13:09:30 2013 -0700 + Merge pull request #928 from jerryshao/fairscheduler-refactor From 15c356c362529347ea87b95e7a6008e0391faceb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 4 Dec 2013 16:41:55 -0800 Subject: [PATCH 14/90] [maven-release-plugin] prepare release spark-parent-0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7e9b201d3546d..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 44acbd9caf20a..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 5e9136fef156c..d2eb3c8c99229 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 2ef2785e17fd3..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a19d8b3b450c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 07213f9d3015f..ec4e766103b30 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + spark-parent-0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index d0e4428908056..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 86d2a0c050436..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index cf5de8f9a4bd9..84f1c7b90b9b5 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d9168e33f6084..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 2dd1e8f8613b7fab409ed8c72b70c48539d54904 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 4 Dec 2013 16:42:46 -0800 Subject: [PATCH 15/90] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..361a1d8c1621c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..2be372da59d5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d2eb3c8c99229..76a01a8cb4a54 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..d12e3abf2e89b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..bd6a1c35f1a8c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index ec4e766103b30..37375adc406c6 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - spark-parent-0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..6f14fe87ae650 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..7a59d89819dd1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 84f1c7b90b9b5..6850ecddc9fcf 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..66e492e76f873 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..002c731851e52 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml From a35e1868662068e9d08b492b0b9f191bb7894d98 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 4 Dec 2013 17:26:32 -0800 Subject: [PATCH 16/90] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 2dd1e8f8613b7fab409ed8c72b70c48539d54904. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 361a1d8c1621c..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 2be372da59d5f..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 76a01a8cb4a54..d2eb3c8c99229 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d12e3abf2e89b..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bd6a1c35f1a8c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 37375adc406c6..ec4e766103b30 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + spark-parent-0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6f14fe87ae650..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 7a59d89819dd1..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 6850ecddc9fcf..84f1c7b90b9b5 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 66e492e76f873..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 002c731851e52..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 27212addd19d2032f9d354b51acf766720cd1c70 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 4 Dec 2013 17:26:36 -0800 Subject: [PATCH 17/90] Revert "[maven-release-plugin] prepare release spark-parent-0.8.1-incubating" This reverts commit 15c356c362529347ea87b95e7a6008e0391faceb. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..7e9b201d3546d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..44acbd9caf20a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d2eb3c8c99229..5e9136fef156c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..2ef2785e17fd3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..2a19d8b3b450c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index ec4e766103b30..07213f9d3015f 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - spark-parent-0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..d0e4428908056 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..86d2a0c050436 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 84f1c7b90b9b5..cf5de8f9a4bd9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..21a6ba981262f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..d9168e33f6084 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml From 47fce43cfddb22b7173d9d12f815a9a05ffd1ca0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 5 Dec 2013 12:31:24 -0800 Subject: [PATCH 18/90] Merge pull request #228 from pwendell/master Document missing configs and set shuffle consolidation to false. (cherry picked from commit 5d460253d6080d871cb71efb112ea17be0873771) Signed-off-by: Patrick Wendell --- .../spark/storage/ShuffleBlockManager.scala | 2 +- .../spark/storage/DiskBlockManagerSuite.scala | 14 ++++++- docs/configuration.md | 37 ++++++++++++++++++- 3 files changed, 49 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 2f1b049ce4839..e828e1d1c5e7b 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -62,7 +62,7 @@ class ShuffleBlockManager(blockManager: BlockManager) { // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. // TODO: Remove this once the shuffle file consolidation feature is stable. val consolidateShuffleFiles = - System.getProperty("spark.shuffle.consolidateFiles", "true").toBoolean + System.getProperty("spark.shuffle.consolidateFiles", "false").toBoolean private val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 0b9056344c1dd..ef4c4c0f143f9 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -5,9 +5,9 @@ import java.io.{FileWriter, File} import scala.collection.mutable import com.google.common.io.Files -import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { +class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { val rootDir0 = Files.createTempDir() rootDir0.deleteOnExit() @@ -16,6 +16,12 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { val rootDirs = rootDir0.getName + "," + rootDir1.getName println("Created root dirs: " + rootDirs) + // This suite focuses primarily on consolidation features, + // so we coerce consolidation if not already enabled. + val consolidateProp = "spark.shuffle.consolidateFiles" + val oldConsolidate = Option(System.getProperty(consolidateProp)) + System.setProperty(consolidateProp, "true") + val shuffleBlockManager = new ShuffleBlockManager(null) { var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]() override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id) @@ -23,6 +29,10 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { var diskBlockManager: DiskBlockManager = _ + override def afterAll() { + oldConsolidate.map(c => System.setProperty(consolidateProp, c)) + } + override def beforeEach() { diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) shuffleBlockManager.idToSegmentMap.clear() diff --git a/docs/configuration.md b/docs/configuration.md index 97183bafdb3ea..22abe1c5a9943 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -327,7 +327,42 @@ Apart from these, the following properties are also available, and may be useful Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit. - + + spark.shuffle.consolidateFiles + false + + If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance if you run shuffles with large numbers of reduce tasks. + + + + + spark.speculation + false + + If set to "true", performs speculative execution of tasks. This means if one or more tasks are running slowly in a stage, they will be re-launched. + + + + spark.speculation.interval + 100 + + How often Spark will check for tasks to speculate, in milliseconds. + + + + spark.speculation.quantile + 0.75 + + Percentage of tasks which must be complete before speculation is enabled for a particular stage. + + + + spark.speculation.multiplier + 1.5 + + How many times slower a task is than the median to be considered for speculation. + + # Environment Variables From d80a4891feaac466a4e05d6bdd2d481a142e03f2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 5 Dec 2013 13:10:19 -0800 Subject: [PATCH 19/90] Bumping version numbers for 0.8.1 release --- docs/_config.yml | 4 ++-- ec2/spark_ec2.py | 4 ++-- project/SparkBuild.scala | 2 +- python/pyspark/shell.py | 2 +- repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index a260922fd00f3..cca5fd93b34cd 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -3,8 +3,8 @@ markdown: kramdown # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 0.8.1-incubating-SNAPSHOT -SPARK_VERSION_SHORT: 0.8.1-SNAPSHOT +SPARK_VERSION: 0.8.1-incubating +SPARK_VERSION_SHORT: 0.8.1 SCALA_VERSION: 2.9.3 MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 267c8ba849821..d508ab379e84d 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -66,7 +66,7 @@ def parse_args(): "slaves across multiple (an additional $0.01/Gb for bandwidth" + "between zones applies)") parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") - parser.add_option("-v", "--spark-version", default="0.8.0", + parser.add_option("-v", "--spark-version", default="0.8.1", help="Version of Spark to use: 'X.Y.Z' or a specific git hash") parser.add_option("--spark-git-repo", default="https://github.com/apache/incubator-spark", @@ -157,7 +157,7 @@ def is_active(instance): # Return correct versions of Spark and Shark, given the supplied Spark version def get_spark_shark_version(opts): - spark_shark_map = {"0.7.3": "0.7.1", "0.8.0": "0.8.0"} + spark_shark_map = {"0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1"} version = opts.spark_version.replace("v", "") if version not in spark_shark_map: print >> stderr, "Don't know about Spark version: %s" % version diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 95a9ca9d5c1c8..660f0e2eff822 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -79,7 +79,7 @@ object SparkBuild extends Build { def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.apache.spark", - version := "0.8.1-incubating-SNAPSHOT", + version := "0.8.1-incubating", scalaVersion := "2.9.3", scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-target:" + SCALAC_JVM_VERSION), diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 59a7e879541bb..2329497e48d78 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -35,7 +35,7 @@ ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /__ / .__/\_,_/_/ /_/\_\ version 0.8.1-SNAPSHOT + /__ / .__/\_,_/_/ /_/\_\ version 0.8.1 /_/ """ print "Using Python version %s (%s, %s)" % ( diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index a5eeacb87b181..6b5be41ce844a 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -200,7 +200,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 0.8.1-SNAPSHOT + /___/ .__/\_,_/_/ /_/\_\ version 0.8.1 /_/ """) import Properties._ From d77c3371b6b189351a2c23cb533777bbd08684c8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 5 Dec 2013 23:29:42 -0800 Subject: [PATCH 20/90] Merge pull request #232 from markhamstra/FiniteWait jobWaiter.synchronized before jobWaiter.wait ...else ``IllegalMonitorStateException`` in ``SimpleFutureAction#ready``. (cherry picked from commit 078049877e123fe7e4c4553e36055de572cab7c4) Signed-off-by: Reynold Xin --- .../scala/org/apache/spark/FutureAction.scala | 2 +- .../apache/spark/scheduler/JobWaiter.scala | 1 + .../spark/rdd/AsyncRDDActionsSuite.scala | 26 +++++++++++++++++++ 3 files changed, 28 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 1ad9240cfa63e..c6b4ac5192d14 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -99,7 +99,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: override def ready(atMost: Duration)(implicit permit: CanAwait): SimpleFutureAction.this.type = { if (!atMost.isFinite()) { awaitResult() - } else { + } else jobWaiter.synchronized { val finishTime = System.currentTimeMillis() + atMost.toMillis while (!isCompleted) { val time = System.currentTimeMillis() diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 58f238d8cfc5d..b026f860a8cd8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -31,6 +31,7 @@ private[spark] class JobWaiter[T]( private var finishedTasks = 0 // Is the job as a whole finished (succeeded or failed)? + @volatile private var _jobFinished = totalTasks == 0 def jobFinished = _jobFinished diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index da032b17d98a6..0d4c10db8ef33 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.rdd import java.util.concurrent.Semaphore +import scala.concurrent.{Await, TimeoutException} +import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global import org.scalatest.{BeforeAndAfterAll, FunSuite} @@ -173,4 +175,28 @@ class AsyncRDDActionsSuite extends FunSuite with BeforeAndAfterAll with Timeouts sem.acquire(2) } } + + /** + * Awaiting FutureAction results + */ + test("FutureAction result, infinite wait") { + val f = sc.parallelize(1 to 100, 4) + .countAsync() + assert(Await.result(f, Duration.Inf) === 100) + } + + test("FutureAction result, finite wait") { + val f = sc.parallelize(1 to 100, 4) + .countAsync() + assert(Await.result(f, Duration(30, "seconds")) === 100) + } + + test("FutureAction result, timeout") { + val f = sc.parallelize(1 to 100, 4) + .mapPartitions(itr => { Thread.sleep(20); itr }) + .countAsync() + intercept[TimeoutException] { + Await.result(f, Duration(20, "milliseconds")) + } + } } From d3a30257a11a1285b0999ad86fec722956bc46e9 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 11 Nov 2013 16:06:12 -0800 Subject: [PATCH 21/90] Added stageId <--> jobId mapping in DAGScheduler ...and make sure that DAGScheduler data structures are cleaned up on job completion. Initial effort and discussion at https://github.com/mesos/spark/pull/842 Conflicts: core/src/main/scala/org/apache/spark/MapOutputTracker.scala core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala --- .../org/apache/spark/MapOutputTracker.scala | 8 +- .../apache/spark/scheduler/DAGScheduler.scala | 305 +++++++++++++----- .../spark/scheduler/DAGSchedulerEvent.scala | 5 +- .../spark/scheduler/SparkListener.scala | 2 +- .../scheduler/cluster/ClusterScheduler.scala | 4 +- .../cluster/ClusterTaskSetManager.scala | 2 +- .../scheduler/local/LocalScheduler.scala | 27 +- .../apache/spark/JobCancellationSuite.scala | 4 +- .../spark/scheduler/DAGSchedulerSuite.scala | 54 ++-- 9 files changed, 299 insertions(+), 112 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1e3f1ebfafaaf..4be338e4c56d0 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -247,7 +247,7 @@ private[spark] class MapOutputTracker extends Logging { case Some(bytes) => return bytes case None => - statuses = mapStatuses(shuffleId) + statuses = mapStatuses.getOrElse(shuffleId, Array[MapStatus]()) epochGotten = epoch } } @@ -261,9 +261,13 @@ private[spark] class MapOutputTracker extends Logging { cachedSerializedStatuses(shuffleId) = bytes } } - return bytes + bytes } + def has(shuffleId: Int): Boolean = { + cachedSerializedStatuses.get(shuffleId).isDefined || mapStatuses.contains(shuffleId) + } + // Serialize an array of map output locations into an efficient byte format so that we can send // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will // generally be pretty compressible because many map outputs will be on the same hostname. diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ec19daf89bf1c..11f80a1dc1f38 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -118,6 +118,10 @@ class DAGScheduler( private val nextStageId = new AtomicInteger(0) + private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]] + + private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]] + private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage] private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] @@ -189,7 +193,7 @@ class DAGScheduler( shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => - val stage = newStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, Some(shuffleDep), jobId) + val stage = newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId) shuffleToMapStage(shuffleDep.shuffleId) = stage stage } @@ -198,7 +202,8 @@ class DAGScheduler( /** * Create a Stage for the given RDD, either as a shuffle map stage (for a ShuffleDependency) or * as a result stage for the final RDD used directly in an action. The stage will also be - * associated with the provided jobId. + * associated with the provided jobId.. Shuffle map stages, whose shuffleId may have previously + * been registered in the MapOutputTracker, should be (re)-created using newOrUsedStage. */ private def newStage( rdd: RDD[_], @@ -208,22 +213,46 @@ class DAGScheduler( 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 - logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") - mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size) - } val id = nextStageId.getAndIncrement() val stage = new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage + registerJobIdWithStages(jobId, stage) stageToInfos(stage) = new StageInfo(stage) val stageIdSet = jobIdToStageIds.getOrElseUpdate(jobId, new HashSet) stageIdSet += id stage } + /** + * Create a shuffle map Stage for the given RDD. The stage will also be associated with the + * provided jobId. If a stage for the shuffleId existed previously so that the shuffleId is + * present in the MapOutputTracker, then the number and location of available outputs are + * recovered from the MapOutputTracker + */ + private def newOrUsedStage( + rdd: RDD[_], + numTasks: Int, + shuffleDep: ShuffleDependency[_,_], + jobId: Int, + callSite: Option[String] = None) + : Stage = + { + val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) + if (mapOutputTracker.has(shuffleDep.shuffleId)) { + val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) + val locs = MapOutputTracker.deserializeMapStatuses(serLocs) + for (i <- 0 until locs.size) stage.outputLocs(i) = List(locs(i)) + stage.numAvailableOutputs = locs.size + } else { + // 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 + logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") + mapOutputTracker.registerShuffle(shuffleDep.shuffleId, rdd.partitions.size) + } + stage + } + /** * Get or create the list of parent stages for a given RDD. The stages will be assigned the * provided jobId if they haven't already been created with a lower jobId. @@ -275,6 +304,91 @@ class DAGScheduler( missing.toList } + /** + * Registers the given jobId among the jobs that need the given stage and + * all of that stage's ancestors. + */ + private def registerJobIdWithStages(jobId: Int, stage: Stage) { + def registerJobIdWithStageList(stages: List[Stage]) { + if (!stages.isEmpty) { + val s = stages.head + stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId + val parents = getParentStages(s.rdd, jobId) + val parentsWithoutThisJobId = parents.filter(p => !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) + registerJobIdWithStageList(parentsWithoutThisJobId ++ stages.tail) + } + } + registerJobIdWithStageList(List(stage)) + } + + private def jobIdToStageIdsAdd(jobId: Int) { + val stageSet = jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) + stageIdToJobIds.foreach { case (stageId, jobSet) => + if (jobSet.contains(jobId)) { + stageSet += stageId + } + } + } + + // Removes job and applies p to any stages that aren't needed by any other jobs + private def forIndependentStagesOfRemovedJob(jobId: Int)(p: Int => Unit) { + val registeredStages = jobIdToStageIds(jobId) + if (registeredStages.isEmpty) { + logError("No stages registered for job " + jobId) + } else { + stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach { + case (stageId, jobSet) => + if (!jobSet.contains(jobId)) { + logError("Job %d not registered for stage %d even though that stage was registered for the job" + .format(jobId, stageId)) + } else { + jobSet -= jobId + if ((jobSet - jobId).isEmpty) { // no other job needs this stage + p(stageId) + } + } + } + } + } + + private def removeStage(stageId: Int) { + // data structures based on Stage + stageIdToStage.get(stageId).foreach { s => + if (running.contains(s)) { + logDebug("Removing running stage %d".format(stageId)) + running -= s + } + stageToInfos -= s + shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleToMapStage.remove(_)) + if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { + logDebug("Removing pending status for stage %d".format(stageId)) + } + pendingTasks -= s + if (waiting.contains(s)) { + logDebug("Removing stage %d from waiting set.".format(stageId)) + waiting -= s + } + if (failed.contains(s)) { + logDebug("Removing stage %d from failed set.".format(stageId)) + failed -= s + } + } + // data structures based on StageId + stageIdToStage -= stageId + stageIdToJobIds -= stageId + + logDebug("After removal of stage %d, remaining stages = %d".format(stageId, stageIdToStage.size)) + } + + private def jobIdToStageIdsRemove(jobId: Int) { + if (!jobIdToStageIds.contains(jobId)) { + logDebug("Trying to remove unregistered job " + jobId) + } else { + forIndependentStagesOfRemovedJob(jobId) { removeStage } + jobIdToStageIds -= jobId + } + } + /** * Submit a job to the job scheduler and get a JobWaiter object back. The JobWaiter object * can be used to block until the the job finishes executing or can be used to cancel the job. @@ -394,35 +508,33 @@ class DAGScheduler( // Compute very short actions like first() or take() with no parent stages locally. runLocally(job) } else { - listenerBus.post(SparkListenerJobStart(job, properties)) idToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job + jobIdToStageIdsAdd(jobId) + listenerBus.post(SparkListenerJobStart(job, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } case JobCancelled(jobId) => - // Cancel a job: find all the running stages that are linked to this job, and cancel them. - running.filter(_.jobId == jobId).foreach { stage => - taskSched.cancelTasks(stage.id) - } + handleJobCancellation(jobId) + idToActiveJob.get(jobId).foreach(job => activeJobs -= job) + idToActiveJob -= jobId case JobGroupCancelled(groupId) => // Cancel all jobs belonging to this job group. // First finds all active jobs with this group id, and then kill stages for them. - val jobIds = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) - .map(_.jobId) - if (!jobIds.isEmpty) { - running.filter(stage => jobIds.contains(stage.jobId)).foreach { stage => - taskSched.cancelTasks(stage.id) - } - } + val activeInGroup = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) + val jobIds = activeInGroup.map(_.jobId) + jobIds.foreach { handleJobCancellation } + activeJobs -- activeInGroup + idToActiveJob -- jobIds case AllJobsCancelled => // Cancel all running jobs. - running.foreach { stage => - taskSched.cancelTasks(stage.id) - } + running.map(_.jobId).foreach { handleJobCancellation } + activeJobs.clear() + idToActiveJob.clear() case ExecutorGained(execId, host) => handleExecutorGained(execId, host) @@ -440,8 +552,18 @@ class DAGScheduler( listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) + case LocalJobCompleted(stage) => + stageIdToJobIds -= stage.id // clean up data structures that were populated for a local job, + stageIdToStage -= stage.id // but that won't get cleaned up via the normal paths through + stageToInfos -= stage // completion events or stage abort + case TaskSetFailed(taskSet, reason) => - abortStage(stageIdToStage.get(taskSet.stageId), reason) + stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } + + case ResubmitFailedStages => + if (failed.size > 0) { + resubmitFailedStages() + } case StopDAGScheduler => // Cancel any active jobs @@ -554,30 +676,52 @@ class DAGScheduler( } catch { case e: Exception => job.listener.jobFailed(e) + } finally { + eventQueue.put(LocalJobCompleted(job.finalStage)) + } + } + + /** Finds the earliest-created active job that needs the stage */ + // TODO: Probably should actually find among the active jobs that need this + // stage the one with the highest priority (highest-priority pool, earliest created). + // That should take care of at least part of the priority inversion problem with + // cross-job dependencies. + private def activeJobForStage(stage: Stage): Option[Int] = { + if (stageIdToJobIds.contains(stage.id)) { + val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted + jobsThatUseStage.find(idToActiveJob.contains(_)) + } else { + None } } /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { - logDebug("submitStage(" + stage + ")") - if (!waiting(stage) && !running(stage) && !failed(stage)) { - val missing = getMissingParentStages(stage).sortBy(_.id) - logDebug("missing: " + missing) - if (missing == Nil) { - logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") - submitMissingTasks(stage) - running += stage - } else { - for (parent <- missing) { - submitStage(parent) + val jobId = activeJobForStage(stage) + if (jobId.isDefined) { + logDebug("submitStage(" + stage + ")") + if (!waiting(stage) && !running(stage) && !failed(stage)) { + val missing = getMissingParentStages(stage).sortBy(_.id) + logDebug("missing: " + missing) + if (missing == Nil) { + logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") + submitMissingTasks(stage, jobId.get) + running += stage + } else { + for (parent <- missing) { + submitStage(parent) + } + waiting += stage } - waiting += stage } + } else { + abortStage(stage, "No active job for stage " + stage.id) } } + /** Called when stage's parents are available and we can now do its task. */ - private def submitMissingTasks(stage: Stage) { + private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry val myPending = pendingTasks.getOrElseUpdate(stage, new HashSet) @@ -598,7 +742,7 @@ class DAGScheduler( } } - val properties = if (idToActiveJob.contains(stage.jobId)) { + val properties = if (idToActiveJob.contains(jobId)) { idToActiveJob(stage.jobId).properties } else { //this stage will be assigned to "default" pool @@ -679,7 +823,7 @@ class DAGScheduler( resultStageToJob -= stage markStageAsFinished(stage) listenerBus.post(SparkListenerJobEnd(job, JobSucceeded)) - removeStages(job) + jobIdToStageIdsRemove(job.jobId) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -715,7 +859,7 @@ class DAGScheduler( changeEpoch = true) } clearCacheLocs() - if (stage.outputLocs.count(_ == Nil) != 0) { + if (stage.outputLocs.exists(_ == Nil)) { // Some tasks had failed; let's resubmit this stage // TODO: Lower-level scheduler should also deal with this logInfo("Resubmitting " + stage + " (" + stage.name + @@ -732,9 +876,12 @@ class DAGScheduler( } waiting --= newlyRunnable running ++= newlyRunnable - for (stage <- newlyRunnable.sortBy(_.id)) { + for { + stage <- newlyRunnable.sortBy(_.id) + jobId <- activeJobForStage(stage) + } { logInfo("Submitting " + stage + " (" + stage.rdd + "), which is now runnable") - submitMissingTasks(stage) + submitMissingTasks(stage, jobId) } } } @@ -819,25 +966,42 @@ class DAGScheduler( } } + private def handleJobCancellation(jobId: Int) { + if (!jobIdToStageIds.contains(jobId)) { + logDebug("Trying to cancel unregistered job " + jobId) + } else { + forIndependentStagesOfRemovedJob(jobId) { stageId => + taskSched.cancelTasks(stageId) + removeStage(stageId) + } + val error = new SparkException("Job %d cancelled".format(jobId)) + val job = idToActiveJob(jobId) + job.listener.jobFailed(error) + listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) + jobIdToStageIds -= jobId + } + } + /** * Aborts all jobs depending on a particular Stage. This is called in response to a task set * being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. */ - private def abortStage(failedStage: Option[Stage], reason: String) { - val dependentStages = if (failedStage.isDefined) - resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage.get)).toSeq else Seq() - failedStage.foreach {stage => - stageToInfos(stage).completionTime = Some(System.currentTimeMillis()) - for (resultStage <- dependentStages) { - val job = resultStageToJob(resultStage) - val error = new SparkException("Job aborted: " + reason) - job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, failedStage))) - idToActiveJob -= resultStage.jobId - activeJobs -= job - resultStageToJob -= resultStage - removeStages(job) - } + private def abortStage(failedStage: Stage, reason: String) { + if (!stageIdToStage.contains(failedStage.id)) { + // Skip all the actions if the stage has been removed. + return + } + val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq + stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) + for (resultStage <- dependentStages) { + val job = resultStageToJob(resultStage) + val error = new SparkException("Job aborted: " + reason) + job.listener.jobFailed(error) + listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) + jobIdToStageIdsRemove(job.jobId) + idToActiveJob -= resultStage.jobId + activeJobs -= job + resultStageToJob -= resultStage } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") @@ -908,21 +1072,18 @@ class DAGScheduler( } private def cleanup(cleanupTime: Long) { - var sizeBefore = stageIdToStage.size - stageIdToStage.clearOldValues(cleanupTime) - logInfo("stageIdToStage " + sizeBefore + " --> " + stageIdToStage.size) - - sizeBefore = shuffleToMapStage.size - shuffleToMapStage.clearOldValues(cleanupTime) - logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size) - - sizeBefore = pendingTasks.size - pendingTasks.clearOldValues(cleanupTime) - logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) - - sizeBefore = stageToInfos.size - stageToInfos.clearOldValues(cleanupTime) - logInfo("stageToInfos " + sizeBefore + " --> " + stageToInfos.size) + Map( + "stageIdToStage" -> stageIdToStage, + "shuffleToMapStage" -> shuffleToMapStage, + "pendingTasks" -> pendingTasks, + "stageToInfos" -> stageToInfos, + "jobIdToStageIds" -> jobIdToStageIds, + "stageIdToJobIds" -> stageIdToJobIds). + foreach { case(s, t) => { + val sizeBefore = t.size + t.clearOldValues(cleanupTime) + logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) + }} } def removeStages(job: ActiveJob) = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 708d221d60caf..e901169adb145 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -65,8 +65,9 @@ private[scheduler] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[scheduler] -case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent +private[scheduler] case class LocalJobCompleted(stage: Stage) extends DAGSchedulerEvent + +private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index a35081f7b10d7..3841b5616dca2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -37,7 +37,7 @@ case class SparkListenerTaskGettingResult( case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvents -case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) +case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], properties: Properties = null) extends SparkListenerEvents case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 27145ccd5f102..281e65abc9d35 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -180,7 +180,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) backend.killTask(tid, execId) } } - tsm.error("Stage %d was cancelled".format(stageId)) + logInfo("Stage %d was cancelled".format(stageId)) + tsm.removeAllRunningTasks() + taskSetFinished(tsm) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala index 4c5eca8537cd6..5afd3f3e0629d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -573,7 +573,7 @@ private[spark] class ClusterTaskSetManager( runningTasks = runningTasksSet.size } - private def removeAllRunningTasks() { + private[cluster] def removeAllRunningTasks() { val numRunningTasks = runningTasksSet.size runningTasksSet.clear() if (parent != null) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala index 2699f0b33e8d3..1c227fefe48d3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala @@ -144,7 +144,8 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: localActor ! KillTask(tid) } } - tsm.error("Stage %d was cancelled".format(stageId)) + logInfo("Stage %d was cancelled".format(stageId)) + taskSetFinished(tsm) } } @@ -192,17 +193,19 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: synchronized { taskIdToTaskSetId.get(taskId) match { case Some(taskSetId) => - val taskSetManager = activeTaskSets(taskSetId) - taskSetTaskIds(taskSetId) -= taskId - - state match { - case TaskState.FINISHED => - taskSetManager.taskEnded(taskId, state, serializedData) - case TaskState.FAILED => - taskSetManager.taskFailed(taskId, state, serializedData) - case TaskState.KILLED => - taskSetManager.error("Task %d was killed".format(taskId)) - case _ => {} + val taskSetManager = activeTaskSets.get(taskSetId) + taskSetManager.foreach { tsm => + taskSetTaskIds(taskSetId) -= taskId + + state match { + case TaskState.FINISHED => + tsm.taskEnded(taskId, state, serializedData) + case TaskState.FAILED => + tsm.taskFailed(taskId, state, serializedData) + case TaskState.KILLED => + tsm.error("Task %d was killed".format(taskId)) + case _ => {} + } } case None => logInfo("Ignoring update from TID " + taskId + " because its task set is gone") diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index d8a0e983b228f..1121e06e2e6cc 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -114,7 +114,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf // Once A is cancelled, job B should finish fairly quickly. assert(jobB.get() === 100) } - +/* test("two jobs sharing the same stage") { // sem1: make sure cancel is issued after some tasks are launched // sem2: make sure the first stage is not finished until cancel is issued @@ -148,7 +148,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf intercept[SparkException] { f1.get() } intercept[SparkException] { f2.get() } } - + */ def testCount() { // Cancel before launching any tasks { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 0f9b9992bd4d6..ee3f8045ad55e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -206,6 +206,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont submit(rdd, Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("local job") { @@ -218,7 +219,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener)) + assert(scheduler.stageToInfos.size === 1) + runEvent(LocalJobCompleted(scheduler.stageToInfos.keys.head)) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("run trivial job w/ dependency") { @@ -227,6 +231,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("cache location preferences w/ dependency") { @@ -239,12 +244,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) complete(taskSet, Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("trivial job failure") { submit(makeRdd(1, Nil), Array(0)) failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted: some failure") + assertDataStructuresEmpty } test("run trivial shuffle") { @@ -260,6 +267,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("run trivial shuffle with fetch failure") { @@ -285,6 +293,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty } test("ignore late map task completions") { @@ -313,6 +322,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) complete(taskSets(1), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) + assertDataStructuresEmpty } test("run trivial shuffle with out-of-band failure and retry") { @@ -329,15 +339,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) - // have hostC complete the resubmitted task - complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === - Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) - complete(taskSets(2), Seq((Success, 42))) - assert(results === Map(0 -> 42)) - } - - test("recursive shuffle failures") { + // have hostC complete the resubmitted task + complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === + Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + complete(taskSets(2), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + assertDataStructuresEmpty + } + + test("recursive shuffle failures") { val shuffleOneRdd = makeRdd(2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) @@ -363,6 +374,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) complete(taskSets(5), Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } test("cached post-shuffle") { @@ -394,6 +406,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) complete(taskSets(4), Seq((Success, 42))) assert(results === Map(0 -> 42)) + assertDataStructuresEmpty } // test("oneGoodJob") { @@ -447,16 +460,19 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345, 0) - private def assertDagSchedulerEmpty(dagScheduler: DAGScheduler) = { - assert(dagScheduler.pendingTasks.isEmpty) - assert(dagScheduler.activeJobs.isEmpty) - assert(dagScheduler.failed.isEmpty) - assert(dagScheduler.jobIdToStageIds.isEmpty) - assert(dagScheduler.stageIdToStage.isEmpty) - assert(dagScheduler.resultStageToJob.isEmpty) - assert(dagScheduler.running.isEmpty) - assert(dagScheduler.shuffleToMapStage.isEmpty) - assert(dagScheduler.waiting.isEmpty) + private def assertDataStructuresEmpty = { + assert(scheduler.pendingTasks.isEmpty) + assert(scheduler.activeJobs.isEmpty) + assert(scheduler.failed.isEmpty) + assert(scheduler.idToActiveJob.isEmpty) + assert(scheduler.jobIdToStageIds.isEmpty) + assert(scheduler.stageIdToJobIds.isEmpty) + assert(scheduler.stageIdToStage.isEmpty) + assert(scheduler.stageToInfos.isEmpty) + assert(scheduler.resultStageToJob.isEmpty) + assert(scheduler.running.isEmpty) + assert(scheduler.shuffleToMapStage.isEmpty) + assert(scheduler.waiting.isEmpty) } } From 1420c7b925fce2fc0f00bb9a39ab60ca2edca878 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 19 Nov 2013 16:59:42 -0800 Subject: [PATCH 22/90] Fixed intended side-effects --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 11f80a1dc1f38..b741f1749afda 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -527,8 +527,8 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) jobIds.foreach { handleJobCancellation } - activeJobs -- activeInGroup - idToActiveJob -- jobIds + activeJobs --= activeInGroup + idToActiveJob --= jobIds case AllJobsCancelled => // Cancel all running jobs. From e9d061741c30bcdbe1e2792765b4ff625b451444 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 20 Nov 2013 15:47:30 -0800 Subject: [PATCH 23/90] Removed redundant residual re: reverted refactoring. --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b741f1749afda..f42fac7e685cf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -343,7 +343,7 @@ class DAGScheduler( .format(jobId, stageId)) } else { jobSet -= jobId - if ((jobSet - jobId).isEmpty) { // no other job needs this stage + if (jobSet.isEmpty) { // no other job needs this stage p(stageId) } } From bf9447853d23376fda6e6a645b77d65b6c2f814a Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 20 Nov 2013 14:49:09 -0800 Subject: [PATCH 24/90] Improved comment --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f42fac7e685cf..af446cdd76340 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -200,10 +200,9 @@ class DAGScheduler( } /** - * Create a Stage for the given RDD, either as a shuffle map stage (for a ShuffleDependency) or - * as a result stage for the final RDD used directly in an action. The stage will also be - * associated with the provided jobId.. Shuffle map stages, whose shuffleId may have previously - * been registered in the MapOutputTracker, should be (re)-created using newOrUsedStage. + * Create a Stage -- either directly for use as a result stage, or as part of the (re)-creation + * of a shuffle map stage in newOrUsedStage. The stage will be associated with the provided + * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage directly. */ private def newStage( rdd: RDD[_], From b7c2f5c42c691e192fcba3b9eafccf793f4b095f Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 22 Nov 2013 10:20:09 -0800 Subject: [PATCH 25/90] Refactoring to make job removal, stage removal, task cancellation clearer --- .../apache/spark/scheduler/DAGScheduler.scala | 76 ++++++++++--------- 1 file changed, 39 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index af446cdd76340..d6e0bddfb2fd8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -329,9 +329,11 @@ class DAGScheduler( } } - // Removes job and applies p to any stages that aren't needed by any other jobs - private def forIndependentStagesOfRemovedJob(jobId: Int)(p: Int => Unit) { + // Removes job and any stages that are not needed by any other job. Returns the set of ids for stages that + // were removed and whose associated tasks may need to be cancelled. + private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { val registeredStages = jobIdToStageIds(jobId) + val independentStages = new HashSet[Int]() if (registeredStages.isEmpty) { logError("No stages registered for job " + jobId) } else { @@ -341,49 +343,51 @@ class DAGScheduler( logError("Job %d not registered for stage %d even though that stage was registered for the job" .format(jobId, stageId)) } else { + def removeStage(stageId: Int) { + // data structures based on Stage + stageIdToStage.get(stageId).foreach { s => + if (running.contains(s)) { + logDebug("Removing running stage %d".format(stageId)) + running -= s + } + stageToInfos -= s + shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleToMapStage.remove) + if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { + logDebug("Removing pending status for stage %d".format(stageId)) + } + pendingTasks -= s + if (waiting.contains(s)) { + logDebug("Removing stage %d from waiting set.".format(stageId)) + waiting -= s + } + if (failed.contains(s)) { + logDebug("Removing stage %d from failed set.".format(stageId)) + failed -= s + } + } + // data structures based on StageId + stageIdToStage -= stageId + stageIdToJobIds -= stageId + + logDebug("After removal of stage %d, remaining stages = %d".format(stageId, stageIdToStage.size)) + } + jobSet -= jobId if (jobSet.isEmpty) { // no other job needs this stage - p(stageId) + independentStages += stageId + removeStage(stageId) } } } } - } - - private def removeStage(stageId: Int) { - // data structures based on Stage - stageIdToStage.get(stageId).foreach { s => - if (running.contains(s)) { - logDebug("Removing running stage %d".format(stageId)) - running -= s - } - stageToInfos -= s - shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleToMapStage.remove(_)) - if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) { - logDebug("Removing pending status for stage %d".format(stageId)) - } - pendingTasks -= s - if (waiting.contains(s)) { - logDebug("Removing stage %d from waiting set.".format(stageId)) - waiting -= s - } - if (failed.contains(s)) { - logDebug("Removing stage %d from failed set.".format(stageId)) - failed -= s - } - } - // data structures based on StageId - stageIdToStage -= stageId - stageIdToJobIds -= stageId - - logDebug("After removal of stage %d, remaining stages = %d".format(stageId, stageIdToStage.size)) + independentStages.toSet } private def jobIdToStageIdsRemove(jobId: Int) { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to remove unregistered job " + jobId) } else { - forIndependentStagesOfRemovedJob(jobId) { removeStage } + removeJobAndIndependentStages(jobId) jobIdToStageIds -= jobId } } @@ -969,10 +973,8 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - forIndependentStagesOfRemovedJob(jobId) { stageId => - taskSched.cancelTasks(stageId) - removeStage(stageId) - } + val independentStages = removeJobAndIndependentStages(jobId) + independentStages.foreach { taskSched.cancelTasks } val error = new SparkException("Job %d cancelled".format(jobId)) val job = idToActiveJob(jobId) job.listener.jobFailed(error) From f2407335a6387d9519c5d9b86717654a28e7967d Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 22 Nov 2013 11:14:39 -0800 Subject: [PATCH 26/90] Cleaned up job cancellation handling --- .../org/apache/spark/scheduler/DAGScheduler.scala | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d6e0bddfb2fd8..8c2d4f7de3e00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -330,7 +330,7 @@ class DAGScheduler( } // Removes job and any stages that are not needed by any other job. Returns the set of ids for stages that - // were removed and whose associated tasks may need to be cancelled. + // were removed. The associated tasks for those stages need to be cancelled if we got here via job cancellation. private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { val registeredStages = jobIdToStageIds(jobId) val independentStages = new HashSet[Int]() @@ -521,8 +521,6 @@ class DAGScheduler( case JobCancelled(jobId) => handleJobCancellation(jobId) - idToActiveJob.get(jobId).foreach(job => activeJobs -= job) - idToActiveJob -= jobId case JobGroupCancelled(groupId) => // Cancel all jobs belonging to this job group. @@ -530,14 +528,12 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) jobIds.foreach { handleJobCancellation } - activeJobs --= activeInGroup - idToActiveJob --= jobIds case AllJobsCancelled => // Cancel all running jobs. running.map(_.jobId).foreach { handleJobCancellation } - activeJobs.clear() - idToActiveJob.clear() + activeJobs.clear() // These should already be empty by this point, + idToActiveJob.clear() // but just in case we lost track of some jobs... case ExecutorGained(execId, host) => handleExecutorGained(execId, host) @@ -980,6 +976,8 @@ class DAGScheduler( job.listener.jobFailed(error) listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) jobIdToStageIds -= jobId + activeJobs -= job + idToActiveJob -= jobId } } From 416cc02def4a529d24306fc039f668d654a1d793 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 22 Nov 2013 13:14:26 -0800 Subject: [PATCH 27/90] Tightly couple stageIdToJobIds and jobIdToStageIds --- .../apache/spark/scheduler/DAGScheduler.scala | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8c2d4f7de3e00..5a9454f704e92 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -216,7 +216,7 @@ class DAGScheduler( val stage = new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage - registerJobIdWithStages(jobId, stage) + updateJobIdStageIdMaps(jobId, stage) stageToInfos(stage) = new StageInfo(stage) val stageIdSet = jobIdToStageIds.getOrElseUpdate(jobId, new HashSet) stageIdSet += id @@ -307,30 +307,24 @@ class DAGScheduler( * Registers the given jobId among the jobs that need the given stage and * all of that stage's ancestors. */ - private def registerJobIdWithStages(jobId: Int, stage: Stage) { - def registerJobIdWithStageList(stages: List[Stage]) { + private def updateJobIdStageIdMaps(jobId: Int, stage: Stage) { + def updateJobIdStageIdMapsList(stages: List[Stage]) { if (!stages.isEmpty) { val s = stages.head stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId + jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id val parents = getParentStages(s.rdd, jobId) val parentsWithoutThisJobId = parents.filter(p => !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) - registerJobIdWithStageList(parentsWithoutThisJobId ++ stages.tail) + updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail) } } - registerJobIdWithStageList(List(stage)) + updateJobIdStageIdMapsList(List(stage)) } - private def jobIdToStageIdsAdd(jobId: Int) { - val stageSet = jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) - stageIdToJobIds.foreach { case (stageId, jobSet) => - if (jobSet.contains(jobId)) { - stageSet += stageId - } - } - } - - // Removes job and any stages that are not needed by any other job. Returns the set of ids for stages that - // were removed. The associated tasks for those stages need to be cancelled if we got here via job cancellation. + /** + * Removes job and any stages that are not needed by any other job. Returns the set of ids for stages that + * were removed. The associated tasks for those stages need to be cancelled if we got here via job cancellation. + */ private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { val registeredStages = jobIdToStageIds(jobId) val independentStages = new HashSet[Int]() @@ -514,7 +508,6 @@ class DAGScheduler( idToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job - jobIdToStageIdsAdd(jobId) listenerBus.post(SparkListenerJobStart(job, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } @@ -552,9 +545,11 @@ class DAGScheduler( handleTaskCompletion(completion) case LocalJobCompleted(stage) => + val jobId = stageIdToJobIds(stage.id).head stageIdToJobIds -= stage.id // clean up data structures that were populated for a local job, stageIdToStage -= stage.id // but that won't get cleaned up via the normal paths through stageToInfos -= stage // completion events or stage abort + jobIdToStageIds -= jobId case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } From 3fd2d40bbe790740938dde1e2a6eacb91734b425 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sun, 24 Nov 2013 17:49:14 -0800 Subject: [PATCH 28/90] Local jobs post SparkListenerJobEnd, and DAGScheduler data structure cleanup always occurs before any posting of SparkListenerJobEnd. Conflicts: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala --- .../apache/spark/scheduler/DAGScheduler.scala | 17 ++++++++++------- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5a9454f704e92..a6d6d3a40cd68 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -544,12 +544,13 @@ class DAGScheduler( listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) - case LocalJobCompleted(stage) => - val jobId = stageIdToJobIds(stage.id).head + case LocalJobCompleted(job, result) => + val stage = job.finalStage stageIdToJobIds -= stage.id // clean up data structures that were populated for a local job, stageIdToStage -= stage.id // but that won't get cleaned up via the normal paths through stageToInfos -= stage // completion events or stage abort - jobIdToStageIds -= jobId + jobIdToStageIds -= job.jobId + listenerBus.post(SparkListenerJobEnd(job, result)) case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } @@ -655,6 +656,7 @@ class DAGScheduler( // Broken out for easier testing in DAGSchedulerSuite. protected def runLocallyWithinThread(job: ActiveJob) { + var jobResult: JobResult = JobSucceeded try { SparkEnv.set(env) val rdd = job.finalStage.rdd @@ -669,9 +671,10 @@ class DAGScheduler( } } catch { case e: Exception => + jobResult = JobFailed(e, Some(job.finalStage)) job.listener.jobFailed(e) } finally { - eventQueue.put(LocalJobCompleted(job.finalStage)) + eventQueue.put(LocalJobCompleted(job, jobResult)) } } @@ -816,8 +819,8 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) - listenerBus.post(SparkListenerJobEnd(job, JobSucceeded)) jobIdToStageIdsRemove(job.jobId) + listenerBus.post(SparkListenerJobEnd(job, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -969,10 +972,10 @@ class DAGScheduler( val error = new SparkException("Job %d cancelled".format(jobId)) val job = idToActiveJob(jobId) job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) jobIdToStageIds -= jobId activeJobs -= job idToActiveJob -= jobId + listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) } } @@ -991,11 +994,11 @@ class DAGScheduler( val job = resultStageToJob(resultStage) val error = new SparkException("Job aborted: " + reason) job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) jobIdToStageIdsRemove(job.jobId) idToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage + listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index e901169adb145..9ad0f1da378a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -65,7 +65,7 @@ private[scheduler] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[scheduler] case class LocalJobCompleted(stage: Stage) extends DAGSchedulerEvent +private[scheduler] case class LocalJobCompleted(job: ActiveJob, result: JobResult) extends DAGSchedulerEvent private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent From 31bc5e538e4318fa9ccd8c0c17ae1cebb5c3f61f Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 26 Nov 2013 14:06:59 -0800 Subject: [PATCH 29/90] Synchronous, inline cleanup after runLocally Conflicts: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala --- .../org/apache/spark/scheduler/DAGScheduler.scala | 15 ++++++--------- .../spark/scheduler/DAGSchedulerEvent.scala | 2 -- .../spark/scheduler/DAGSchedulerSuite.scala | 2 -- 3 files changed, 6 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index a6d6d3a40cd68..357fd8f82e309 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -544,14 +544,6 @@ class DAGScheduler( listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) - case LocalJobCompleted(job, result) => - val stage = job.finalStage - stageIdToJobIds -= stage.id // clean up data structures that were populated for a local job, - stageIdToStage -= stage.id // but that won't get cleaned up via the normal paths through - stageToInfos -= stage // completion events or stage abort - jobIdToStageIds -= job.jobId - listenerBus.post(SparkListenerJobEnd(job, result)) - case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } @@ -674,7 +666,12 @@ class DAGScheduler( jobResult = JobFailed(e, Some(job.finalStage)) job.listener.jobFailed(e) } finally { - eventQueue.put(LocalJobCompleted(job, jobResult)) + val s = job.finalStage + stageIdToJobIds -= s.id // clean up data structures that were populated for a local job, + stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through + stageToInfos -= s // completion events or stage abort + jobIdToStageIds -= job.jobId + listenerBus.post(SparkListenerJobEnd(job, jobResult)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 9ad0f1da378a6..9713a889cda45 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -65,8 +65,6 @@ private[scheduler] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[scheduler] case class LocalJobCompleted(job: ActiveJob, result: JobResult) extends DAGSchedulerEvent - private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ee3f8045ad55e..03fc8c020e005 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -219,8 +219,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener)) - assert(scheduler.stageToInfos.size === 1) - runEvent(LocalJobCompleted(scheduler.stageToInfos.keys.head)) assert(results === Map(0 -> 42)) assertDataStructuresEmpty } From 03ceccb9228af15d27e62be31d051ee56ae4ec08 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 26 Nov 2013 22:25:20 -0800 Subject: [PATCH 30/90] SparkListenerJobStart posted from local jobs --- .../src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 357fd8f82e309..695be8ac8cee2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -503,6 +503,7 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { // Compute very short actions like first() or take() with no parent stages locally. + listenerBus.post(SparkListenerJobStart(job, Array(), properties)) runLocally(job) } else { idToActiveJob(jobId) = job From 8e247adf0711c8e485be662e0e1e290ab129d692 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 6 Dec 2013 15:23:07 -0800 Subject: [PATCH 31/90] Merge fix --- .../org/apache/spark/scheduler/DAGScheduler.scala | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 695be8ac8cee2..2ecc48a6b0566 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -114,8 +114,6 @@ class DAGScheduler( def numTotalJobs: Int = nextJobId.get() - val jobIdToStageIds = new HashMap[Int, HashSet[Int]] - private val nextStageId = new AtomicInteger(0) private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]] @@ -239,8 +237,8 @@ class DAGScheduler( { val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) if (mapOutputTracker.has(shuffleDep.shuffleId)) { - val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) - val locs = MapOutputTracker.deserializeMapStatuses(serLocs) + val serLocs = mapOutputTracker.getSerializedLocations(shuffleDep.shuffleId) + val locs = mapOutputTracker.deserializeStatuses(serLocs) for (i <- 0 until locs.size) stage.outputLocs(i) = List(locs(i)) stage.numAvailableOutputs = locs.size } else { @@ -548,11 +546,6 @@ class DAGScheduler( case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } - case ResubmitFailedStages => - if (failed.size > 0) { - resubmitFailedStages() - } - case StopDAGScheduler => // Cancel any active jobs for (job <- activeJobs) { @@ -757,7 +750,7 @@ class DAGScheduler( SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) } catch { case e: NotSerializableException => - abortStage(Some(stage), "Task not serializable: " + e.toString) + abortStage(stage, "Task not serializable: " + e.toString) running -= stage return } @@ -920,7 +913,7 @@ class DAGScheduler( case other => // Unrecognized failure - abort all jobs depending on this stage - abortStage(stageIdToStage.get(task.stageId), task + " failed: " + other) + abortStage(stageIdToStage(task.stageId), task + " failed: " + other) } } From ee22be0e6c302fb2cdb24f83365c2b8a43a1baab Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 19 Nov 2013 16:05:44 -0800 Subject: [PATCH 32/90] Merge pull request #189 from tgravescs/sparkYarnErrorHandling Impove Spark on Yarn Error handling Improve cli error handling and only allow a certain number of worker failures before failing the application. This will help prevent users from doing foolish things and their jobs running forever. For instance using 32 bit java but trying to allocate 8G containers. This loops forever without this change, now it errors out after a certain number of retries. The number of tries is configurable. Also increase the frequency we ping the RM to increase speed at which we get containers if they die. The Yarn MR app defaults to pinging the RM every 1 seconds, so the default of 5 seconds here is fine. But that is configurable as well in case people want to change it. I do want to make sure there aren't any cases that calling stopExecutors in CoarseGrainedSchedulerBackend would cause problems? I couldn't think of any and testing on standalone cluster as well as yarn. (cherry picked from commit aa638ed9c140174a47df082ed5631ffe8e624ee6) Signed-off-by: Patrick Wendell --- .../CoarseGrainedSchedulerBackend.scala | 1 + .../cluster/SimrSchedulerBackend.scala | 1 - docs/running-on-yarn.md | 2 + .../spark/deploy/yarn/ApplicationMaster.scala | 39 ++++++++++++------- .../org/apache/spark/deploy/yarn/Client.scala | 32 +++++++++------ .../deploy/yarn/YarnAllocationHandler.scala | 16 ++++++-- 6 files changed, 61 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index a45bee536ca9d..d0ba5bf55dcfd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -199,6 +199,7 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac } override def stop() { + stopExecutors() try { if (driverActor != null) { val future = driverActor.ask(StopDriver)(timeout) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index 0ea35e2b7a311..e000531a26f7e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -62,7 +62,6 @@ private[spark] class SimrSchedulerBackend( val conf = new Configuration() val fs = FileSystem.get(conf) fs.delete(new Path(driverFilePath), false) - super.stopExecutors() super.stop() } } diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 6fd1d0d150306..4056e9c15db2b 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -37,6 +37,8 @@ System Properties: * 'spark.yarn.applicationMaster.waitTries', property to set the number of times the ApplicationMaster waits for the the spark master and then also the number of tries it waits for the Spark Context to be intialized. Default is 10. * 'spark.yarn.submit.file.replication', the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives. * 'spark.yarn.preserve.staging.files', set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them. +* 'spark.yarn.scheduler.heartbeat.interval-ms', the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds. +* 'spark.yarn.max.worker.failures', the maximum number of worker failures before failing the application. Default is the number of workers requested times 2 with minimum of 3. # Launching Spark on YARN diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 4302ef4cda261..2afc1d97a9cc2 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -54,7 +54,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true - + // default to numWorkers * 2, with minimum of 3 + private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", + math.max(args.numWorkers * 2, 3).toString()).toInt def run() { // setup the directories so things go to yarn approved directories rather @@ -227,12 +229,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e if (null != sparkContext) { uiAddress = sparkContext.ui.appUIAddress - this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, - sparkContext.preferredNodeLocationData) + this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, + appAttemptId, args, sparkContext.preferredNodeLocationData) } else { logWarning("Unable to retrieve sparkContext inspite of waiting for " + count * waitTime + - ", numTries = " + numTries) - this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args) + ", numTries = " + numTries) + this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, + appAttemptId, args) } } } finally { @@ -251,8 +254,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e while(yarnAllocator.getNumWorkersRunning < args.numWorkers && // If user thread exists, then quit ! userThread.isAlive) { - - this.yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of worker failures reached") + } + yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) ApplicationMaster.incrementAllocatorLoop(1) Thread.sleep(100) } @@ -268,21 +274,27 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) - // must be <= timeoutInterval/ 2. - // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. - // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. - val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L)) + + // we want to be reasonably responsive without causing too many requests to RM. + val schedulerInterval = + System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + + // must be <= timeoutInterval / 2. + val interval = math.min(timeoutInterval / 2, schedulerInterval) launchReporterThread(interval) } } - // TODO: We might want to extend this to allocate more containers in case they die ! private def launchReporterThread(_sleepTime: Long): Thread = { val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime val t = new Thread { override def run() { while (userThread.isAlive) { + if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of worker failures reached") + } val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning if (missingWorkerCount > 0) { logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") @@ -321,7 +333,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } */ - def finishApplicationMaster(status: FinalApplicationStatus) { + def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") { synchronized { if (isFinished) { @@ -335,6 +347,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e .asInstanceOf[FinishApplicationMasterRequest] finishReq.setAppAttemptId(appAttemptId) finishReq.setFinishApplicationStatus(status) + finishReq.setDiagnostics(diagnostics) // set tracking url to empty since we don't have a history server finishReq.setTrackingUrl("") resourceManager.finishApplicationMaster(finishReq) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 4e0e060ddc29b..15b3480e02342 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -60,6 +60,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) def run() { + validateArgs() + init(yarnConf) start() logClusterResourceDetails() @@ -84,6 +86,23 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl System.exit(0) } + def validateArgs() = { + Map((System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", + (args.userJar == null) -> "Error: You must specify a user jar!", + (args.userClass == null) -> "Error: You must specify a user class!", + (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!", + (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> + ("Error: AM memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD), + (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> + ("Error: Worker memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString())) + .foreach { case(cond, errStr) => + if (cond) { + logError(errStr) + args.printUsageAndExit(1) + } + } + } + def getAppStagingDir(appId: ApplicationId): String = { SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR } @@ -97,7 +116,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl ", queueMaxCapacity=" + queueInfo.getMaximumCapacity + ", queueApplicationCount=" + queueInfo.getApplications.size + ", queueChildQueueCount=" + queueInfo.getChildQueues.size) } - def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() @@ -215,11 +233,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - if (System.getenv("SPARK_JAR") == null || args.userJar == null) { - logError("Error: You must set SPARK_JAR environment variable and specify a user jar!") - System.exit(1) - } - Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF")) .foreach { case(destName, _localPath) => @@ -334,7 +347,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " - // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out. // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same // node, spark gc effects all other containers performance (which can also be other spark containers) @@ -360,11 +372,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl javaCommand = Environment.JAVA_HOME.$() + "/bin/java" } - if (args.userClass == null) { - logError("Error: You must specify a user class!") - System.exit(1) - } - val commands = List[String](javaCommand + " -server " + JAVA_OPTS + @@ -442,6 +449,7 @@ object Client { System.setProperty("SPARK_YARN_MODE", "true") val args = new ClientArguments(argStrings) + new Client(args).run } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 25da9aa917d95..507a0743fd77a 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -72,9 +72,11 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM // Used to generate a unique id per worker private val workerIdCounter = new AtomicInteger() private val lastResponseId = new AtomicInteger() + private val numWorkersFailed = new AtomicInteger() def getNumWorkersRunning: Int = numWorkersRunning.intValue + def getNumWorkersFailed: Int = numWorkersFailed.intValue def isResourceConstraintSatisfied(container: Container): Boolean = { container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) @@ -253,8 +255,16 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM else { // simply decrement count - next iteration of ReporterThread will take care of allocating ! numWorkersRunning.decrementAndGet() - logInfo("Container completed ? nodeId: " + containerId + ", state " + completedContainer.getState + - " httpaddress: " + completedContainer.getDiagnostics) + logInfo("Container completed not by us ? nodeId: " + containerId + ", state " + completedContainer.getState + + " httpaddress: " + completedContainer.getDiagnostics + " exit status: " + completedContainer.getExitStatus()) + + // Hadoop 2.2.X added a ContainerExitStatus we should switch to use + // there are some exit status' we shouldn't necessarily count against us, but for + // now I think its ok as none of the containers are expected to exit + if (completedContainer.getExitStatus() != 0) { + logInfo("Container marked as failed: " + containerId) + numWorkersFailed.incrementAndGet() + } } allocatedHostToContainersMap.synchronized { @@ -378,8 +388,6 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM val releasedContainerList = createReleasedContainerList() req.addAllReleases(releasedContainerList) - - if (numWorkers > 0) { logInfo("Allocating " + numWorkers + " worker containers with " + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + " of memory each.") } From 2b7631555ca0e8a6812bcbf008c6a719bbd5b4b7 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 16 Nov 2013 11:44:10 -0800 Subject: [PATCH 33/90] Merge pull request #178 from hsaputra/simplecleanupcode Simple cleanup on Spark's Scala code Simple cleanup on Spark's Scala code while testing some modules: -) Remove some of unused imports as I found them -) Remove ";" in the imports statements -) Remove () at the end of method calls like size that does not have size effect. (cherry picked from commit 1b5b358309a5adfc12b75b0ebb4254ad8e69f5a0) Signed-off-by: Patrick Wendell --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 - .../scala/org/apache/spark/deploy/LocalSparkCluster.scala | 6 +++--- .../spark/executor/CoarseGrainedExecutorBackend.scala | 2 +- .../scala/org/apache/spark/executor/ExecutorSource.scala | 2 -- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 8 +++----- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +- .../spark/deploy/yarn/ClientDistributedCacheManager.scala | 2 +- .../deploy/yarn/ClientDistributedCacheManagerSuite.scala | 2 +- 8 files changed, 10 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ff5b2e064a531..9ccf70f828bbb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.Map import scala.collection.generic.Growable -import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 308a2bfa222c3..a724900943108 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -17,12 +17,12 @@ package org.apache.spark.deploy -import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.actor.ActorSystem import org.apache.spark.deploy.worker.Worker import org.apache.spark.deploy.master.Master -import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.{Logging} +import org.apache.spark.util.Utils +import org.apache.spark.Logging import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index caee6b01ab1fa..8332631838d0a 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} -import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.Logging import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 18c9dc1c0a9bb..a537b2ceb1142 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -20,8 +20,6 @@ package org.apache.spark.executor import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.hdfs.DistributedFileSystem -import org.apache.hadoop.fs.LocalFileSystem import scala.collection.JavaConversions._ diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 2afc1d97a9cc2..89b00415daa8e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -17,9 +17,8 @@ package org.apache.spark.deploy.yarn -import java.io.IOException; +import java.io.IOException import java.net.Socket -import java.security.PrivilegedExceptionAction import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import org.apache.hadoop.conf.Configuration @@ -34,7 +33,6 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{SparkContext, Logging} import org.apache.spark.util.Utils -import org.apache.hadoop.security.UserGroupInformation import scala.collection.JavaConversions._ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { @@ -188,8 +186,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e var successed = false try { // Copy - var mainArgs: Array[String] = new Array[String](args.userArgs.size()) - args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) + var mainArgs: Array[String] = new Array[String](args.userArgs.size) + args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size) mainMethod.invoke(null, mainArgs) // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR // userThread will stop here unless it has uncaught exception thrown out diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 15b3480e02342..1078d5b826f67 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil} -import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.mapred.Master import org.apache.hadoop.net.NetUtils import org.apache.hadoop.io.DataOutputBuffer diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 07686fefd7c06..674c8f8112b86 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import java.net.URI; +import java.net.URI import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileStatus diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala index c0a2af0c6faf3..2941356bc55f9 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import java.net.URI; +import java.net.URI import org.scalatest.FunSuite import org.scalatest.mock.MockitoSugar From 20d1f8b4551b5d7e4df37248d3183131119cbd22 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 20 Nov 2013 10:36:10 -0800 Subject: [PATCH 34/90] Merge pull request #191 from hsaputra/removesemicolonscala Cleanup to remove semicolons (;) from Scala code -) The main reason for this PR is to remove semicolons from single statements of Scala code. -) Remove unused imports as I see them -) Fix ASF comment header from some of files (bad copy paste I suppose) (cherry picked from commit 4b895013cc965b37d44fd255656da470a3d2c222) Conflicts: examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala Squash into 191 --- .../spark/deploy/FaultToleranceTest.scala | 28 +++++++++--------- .../spark/network/netty/ShuffleCopier.scala | 2 +- .../org/apache/spark/rdd/CartesianRDD.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../org/apache/spark/LocalSparkContext.scala | 2 +- .../org/apache/spark/PartitioningSuite.scala | 10 +++---- .../org/apache/spark/examples/LocalALS.scala | 2 +- .../org/apache/spark/examples/SparkTC.scala | 2 +- .../streaming/examples/ActorWordCount.scala | 2 +- .../apache/spark/streaming/Checkpoint.scala | 6 ++-- .../api/java/JavaStreamingContext.scala | 7 ++--- .../streaming/dstream/FlumeInputDStream.scala | 4 +-- .../spark/streaming/InputStreamsSuite.scala | 4 +-- .../spark/streaming/TestSuiteBase.scala | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 6 ++-- .../org/apache/spark/deploy/yarn/Client.scala | 29 +++++++++---------- .../yarn/ClientDistributedCacheManager.scala | 4 +-- .../spark/deploy/yarn/WorkerRunnable.scala | 13 ++++----- .../deploy/yarn/YarnSparkHadoopUtil.scala | 5 +--- 19 files changed, 64 insertions(+), 68 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 668032a3a2680..0aa8852649e05 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -1,19 +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. + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. * */ diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index 481ff8c3e02d2..b1e1576dadc1a 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -76,7 +76,7 @@ private[spark] object ShuffleCopier extends Logging { extends FileClientHandler with Logging { override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { - logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)"); + logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)") resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 9b0c882481cfc..0de22f0e06e49 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -70,7 +70,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( override def compute(split: Partition, context: TaskContext) = { val currSplit = split.asInstanceOf[CartesianPartition] for (x <- rdd1.iterator(currSplit.s1, context); - y <- rdd2.iterator(currSplit.s2, context)) yield (x, y) + y <- rdd2.iterator(currSplit.s2, context)) yield (x, y) } override def getDependencies: Seq[Dependency[_]] = List( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index c8625716e9d8e..baccc4281a06c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -133,7 +133,7 @@ private[spark] class StagePage(parent: JobProgressUI) { summary ++

Summary Metrics for {numCompleted} Completed Tasks

++
{summaryTable.getOrElse("No tasks have reported metrics yet.")}
++ -

Tasks

++ taskTable; +

Tasks

++ taskTable headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages) } diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 6ec124da9c7b1..03f7c0b43a71e 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -30,7 +30,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self @transient var sc: SparkContext = _ override def beforeAll() { - InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory()); + InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory()) super.beforeAll() } diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 7d938917f2650..1374d01774693 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -142,11 +142,11 @@ class PartitioningSuite extends FunSuite with SharedSparkContext { .filter(_ >= 0.0) // Run the partitions, including the consecutive empty ones, through StatCounter - val stats: StatCounter = rdd.stats(); - assert(abs(6.0 - stats.sum) < 0.01); - assert(abs(6.0/2 - rdd.mean) < 0.01); - assert(abs(1.0 - rdd.variance) < 0.01); - assert(abs(1.0 - rdd.stdev) < 0.01); + val stats: StatCounter = rdd.stats() + assert(abs(6.0 - stats.sum) < 0.01) + assert(abs(6.0/2 - rdd.mean) < 0.01) + assert(abs(1.0 - rdd.variance) < 0.01) + assert(abs(1.0 - rdd.stdev) < 0.01) // Add other tests here for classes that should be able to handle empty partitions correctly } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 4af45b2b4a067..83db8b9e26411 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -120,7 +120,7 @@ object LocalALS { System.exit(1) } } - printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS); + printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 5a7a9d1bd8f74..8543ce0e3285e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -65,7 +65,7 @@ object SparkTC { oldCount = nextCount // Perform the join, obtaining an RDD of (y, (z, x)) pairs, // then project the result to obtain the new (x, z) paths. - tc = tc.union(tc.join(edges).map(x => (x._2._2, x._2._1))).distinct().cache(); + tc = tc.union(tc.join(edges).map(x => (x._2._2, x._2._1))).distinct().cache() nextCount = tc.count() } while (nextCount != oldCount) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index cd3423a07b6b7..af52b7e9a12f1 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -120,7 +120,7 @@ object FeederActor { println("Feeder started as:" + feeder) - actorSystem.awaitTermination(); + actorSystem.awaitTermination() } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 2d8f07262426f..74069860c0e6e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -92,7 +92,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging { fs.delete(file, false) fs.rename(writeFile, file) - val finishTime = System.currentTimeMillis(); + val finishTime = System.currentTimeMillis() logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds") return @@ -122,7 +122,9 @@ class CheckpointWriter(checkpointDir: String) extends Logging { def stop() { synchronized { - if (stopped) return ; + if (stopped) { + return + } stopped = true } executor.shutdown() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index cfc1c2628cb13..4f6d479401d61 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.api.java -import java.lang.{Long => JLong, Integer => JInt} +import java.lang.{Integer => JInt} import java.io.InputStream import java.util.{Map => JMap, List => JList} @@ -33,10 +33,9 @@ import twitter4j.auth.Authorization import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaSparkContext, JavaRDD} +import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD} import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receivers.{ActorReceiver, ReceiverSupervisorStrategy} /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -302,7 +301,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]] implicit val cmf: ClassManifest[F] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[F]] - ssc.fileStream[K, V, F](directory); + ssc.fileStream[K, V, F](directory) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala index 18de772946805..a0189eca043c6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala @@ -137,8 +137,8 @@ class FlumeReceiver( protected override def onStart() { val responder = new SpecificResponder( - classOf[AvroSourceProtocol], new FlumeEventServer(this)); - val server = new NettyServer(responder, new InetSocketAddress(host, port)); + classOf[AvroSourceProtocol], new FlumeEventServer(this)) + val server = new NettyServer(responder, new InetSocketAddress(host, port)) blockGenerator.start() server.start() logInfo("Flume receiver started") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index ca2da6816a139..f398263a10da5 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -124,9 +124,9 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val input = Seq(1, 2, 3, 4, 5) Thread.sleep(1000) - val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)); + val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)) val client = SpecificRequestor.getClient( - classOf[AvroSourceProtocol], transceiver); + classOf[AvroSourceProtocol], transceiver) for (i <- 0 until input.size) { val event = new AvroFlumeEvent diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index be140699c2964..8c8c359e6e865 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -251,7 +251,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { Thread.sleep(500) // Give some time for the forgetting old RDDs to complete } catch { - case e: Exception => e.printStackTrace(); throw e; + case e: Exception => {e.printStackTrace(); throw e} } finally { ssc.stop() } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 89b00415daa8e..a7baf0c36cfd4 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -21,6 +21,7 @@ import java.io.IOException import java.net.Socket import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.net.NetUtils @@ -33,6 +34,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{SparkContext, Logging} import org.apache.spark.util.Utils + import scala.collection.JavaConversions._ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { @@ -65,7 +67,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) appAttemptId = getApplicationAttemptId() - isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts; + isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts resourceManager = registerWithResourceManager() // Workaround until hadoop moves to something which has @@ -195,7 +197,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e successed = true } finally { logDebug("finishing main") - isLastAMRetry = true; + isLastAMRetry = true if (successed) { ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } else { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 1078d5b826f67..94e353af2e954 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -17,14 +17,13 @@ package org.apache.spark.deploy.yarn -import java.net.{InetAddress, InetSocketAddress, UnknownHostException, URI} +import java.net.{InetAddress, UnknownHostException, URI} import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil} import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.mapred.Master -import org.apache.hadoop.net.NetUtils import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ @@ -40,9 +39,7 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.Map import scala.collection.JavaConversions._ -import org.apache.spark.Logging -import org.apache.spark.util.Utils -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.Logging class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { @@ -123,7 +120,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // if we have requested more then the clusters max for a single resource then exit. if (args.workerMemory > maxMem) { - logError("the worker size is to large to run on this cluster " + args.workerMemory); + logError("the worker size is to large to run on this cluster " + args.workerMemory) System.exit(1) } val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD @@ -160,8 +157,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl var dstHost = dstUri.getHost() if ((srcHost != null) && (dstHost != null)) { try { - srcHost = InetAddress.getByName(srcHost).getCanonicalHostName(); - dstHost = InetAddress.getByName(dstHost).getCanonicalHostName(); + srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() + dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() } catch { case e: UnknownHostException => return false @@ -178,7 +175,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl if (srcUri.getPort() != dstUri.getPort()) { return false } - return true; + return true } /** @@ -190,13 +187,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl replication: Short, setPerms: Boolean = false): Path = { val fs = FileSystem.get(conf) - val remoteFs = originalPath.getFileSystem(conf); + val remoteFs = originalPath.getFileSystem(conf) var newPath = originalPath if (! compareFs(remoteFs, fs)) { newPath = new Path(dstDir, originalPath.getName()) logInfo("Uploading " + originalPath + " to " + newPath) - FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf); - fs.setReplication(newPath, replication); + FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) + fs.setReplication(newPath, replication) if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) } // resolve any symlinks in the URI path so using a "current" symlink @@ -214,7 +211,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Add them as local resources to the AM val fs = FileSystem.get(conf) - val delegTokenRenewer = Master.getMasterPrincipal(conf); + val delegTokenRenewer = Master.getMasterPrincipal(conf) if (UserGroupInformation.isSecurityEnabled()) { if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { logError("Can't get Master Kerberos principal for use as renewer") @@ -226,7 +223,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl if (UserGroupInformation.isSecurityEnabled()) { val dstFs = dst.getFileSystem(conf) - dstFs.addDelegationTokens(delegTokenRenewer, credentials); + dstFs.addDelegationTokens(delegTokenRenewer, credentials) } val localResources = HashMap[String, LocalResource]() FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) @@ -286,7 +283,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } } - UserGroupInformation.getCurrentUser().addCredentials(credentials); + UserGroupInformation.getCurrentUser().addCredentials(credentials) return localResources } @@ -366,7 +363,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } // Command for the ApplicationMaster - var javaCommand = "java"; + var javaCommand = "java" val javaHome = System.getenv("JAVA_HOME") if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { javaCommand = Environment.JAVA_HOME.$() + "/bin/java" diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 674c8f8112b86..5f159b073f537 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -197,11 +197,11 @@ class ClientDistributedCacheManager() extends Logging { */ def checkPermissionOfOther(fs: FileSystem, path: Path, action: FsAction, statCache: Map[URI, FileStatus]): Boolean = { - val status = getFileStatus(fs, path.toUri(), statCache); + val status = getFileStatus(fs, path.toUri(), statCache) val perms = status.getPermission() val otherAction = perms.getOtherAction() if (otherAction.implies(action)) { - return true; + return true } return false } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala index 7a66532254c74..a4d6e1d87d127 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import java.security.PrivilegedExceptionAction import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation @@ -38,7 +38,6 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap import org.apache.spark.Logging -import org.apache.spark.util.Utils class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String, slaveId: String, hostname: String, workerMemory: Int, workerCores: Int) @@ -108,7 +107,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S credentials.writeTokenStorageToStream(dob) ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - var javaCommand = "java"; + var javaCommand = "java" val javaHome = System.getenv("JAVA_HOME") if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { javaCommand = Environment.JAVA_HOME.$() + "/bin/java" @@ -204,8 +203,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S // use doAs and remoteUser here so we can add the container token and not // pollute the current users credentials with all of the individual container tokens - val user = UserGroupInformation.createRemoteUser(container.getId().toString()); - val containerToken = container.getContainerToken(); + val user = UserGroupInformation.createRemoteUser(container.getId().toString()) + val containerToken = container.getContainerToken() if (containerToken != null) { user.addToken(ProtoUtils.convertFromProtoFormat(containerToken, cmAddress)) } @@ -216,8 +215,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S return rpc.getProxy(classOf[ContainerManager], cmAddress, conf).asInstanceOf[ContainerManager] } - }); - return proxy; + }) + proxy } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index ca2f1e2565b9a..2ba2366ead171 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -18,13 +18,10 @@ package org.apache.spark.deploy.yarn import org.apache.spark.deploy.SparkHadoopUtil -import collection.mutable.HashMap import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import java.security.PrivilegedExceptionAction /** * Contains util methods to interact with Hadoop from spark. @@ -40,7 +37,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster override def addCredentials(conf: JobConf) { - val jobCreds = conf.getCredentials(); + val jobCreds = conf.getCredentials() jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) } } From 1e9d08408a964ff0ad2db65c9f67040bac00c284 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 25 Nov 2013 15:25:29 -0800 Subject: [PATCH 35/90] Merge pull request #101 from colorant/yarn-client-scheduler For SPARK-527, Support spark-shell when running on YARN sync to trunk and resubmit here In current YARN mode approaching, the application is run in the Application Master as a user program thus the whole spark context is on remote. This approaching won't support application that involve local interaction and need to be run on where it is launched. So In this pull request I have a YarnClientClusterScheduler and backend added. With this scheduler, the user application is launched locally,While the executor will be launched by YARN on remote nodes with a thin AM which only launch the executor and monitor the Driver Actor status, so that when client app is done, it can finish the YARN Application as well. This enables spark-shell to run upon YARN. This also enable other Spark applications to have the spark context to run locally with a master-url "yarn-client". Thus e.g. SparkPi could have the result output locally on console instead of output in the log of the remote machine where AM is running on. Docs also updated to show how to use this yarn-client mode. (cherry picked from commit eb4296c8f7561aaf8782479dd5cd7c9320b7fa6b) Conflicts: core/src/main/scala/org/apache/spark/SparkContext.scala --- .../scala/org/apache/spark/SparkContext.scala | 25 ++ docs/running-on-yarn.md | 27 +- .../org/apache/spark/deploy/yarn/Client.scala | 13 +- .../spark/deploy/yarn/ClientArguments.scala | 40 +-- .../spark/deploy/yarn/WorkerLauncher.scala | 246 ++++++++++++++++++ .../cluster/YarnClientClusterScheduler.scala | 47 ++++ .../cluster/YarnClientSchedulerBackend.scala | 109 ++++++++ 7 files changed, 484 insertions(+), 23 deletions(-) create mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala create mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala create mode 100644 yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9ccf70f828bbb..82be5581d6b11 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -226,6 +226,31 @@ class SparkContext( scheduler.initialize(backend) scheduler + case "yarn-client" => + val scheduler = try { + val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") + val cons = clazz.getConstructor(classOf[SparkContext]) + cons.newInstance(this).asInstanceOf[ClusterScheduler] + + } catch { + case th: Throwable => { + throw new SparkException("YARN mode not available ?", th) + } + } + + val backend = try { + val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") + val cons = clazz.getConstructor(classOf[ClusterScheduler], classOf[SparkContext]) + cons.newInstance(scheduler, this).asInstanceOf[CoarseGrainedSchedulerBackend] + } catch { + case th: Throwable => { + throw new SparkException("YARN mode not available ?", th) + } + } + + scheduler.initialize(backend) + scheduler + case _ => if (MESOS_REGEX.findFirstIn(master).isEmpty) { logWarning("Master %s does not match expected format, parsing as Mesos URL".format(master)) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 4056e9c15db2b..68fd6c2ab1db2 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -45,6 +45,10 @@ System Properties: Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the hadoop cluster. This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager. +There are two scheduler mode that can be used to launch spark application on YARN. + +## Launch spark application by YARN Client with yarn-standalone mode. + The command to launch the YARN Client is as follows: SPARK_JAR= ./spark-class org.apache.spark.deploy.yarn.Client \ @@ -52,6 +56,7 @@ The command to launch the YARN Client is as follows: --class \ --args \ --num-workers \ + --master-class --master-memory \ --worker-memory \ --worker-cores \ @@ -85,11 +90,29 @@ For example: $ cat $YARN_APP_LOGS_DIR/$YARN_APP_ID/container*_000001/stdout Pi is roughly 3.13794 -The above starts a YARN Client programs which periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running. +The above starts a YARN Client programs which start the default Application Master. Then SparkPi will be run as a child thread of Application Master, YARN Client will periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running. + +With this mode, your application is actually run on the remote machine where the Application Master is run upon. Thus application that involve local interaction will not work well, e.g. spark-shell. + +## Launch spark application with yarn-client mode. + +With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR and SPARK_YARN_APP_JAR + +In order to tune worker core/number/memory etc. You need to export SPARK_WORKER_CORES, SPARK_WORKER_MEMORY, SPARK_WORKER_INSTANCES e.g. by ./conf/spark-env.sh + +For example: + + SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ + SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + ./run-example org.apache.spark.examples.SparkPi yarn-client + + + SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ + SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + MASTER=yarn-client ./spark-shell # Important Notes -- When your application instantiates a Spark context it must use a special "yarn-standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "yarn-standalone" as an argument to your program, as shown in the example above. - We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed. - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored. - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 94e353af2e954..bb73f6d337ba0 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -54,9 +54,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short) // app files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) + val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) - def run() { + // for client user who want to monitor app status by itself. + def runApp() = { validateArgs() init(yarnConf) @@ -78,7 +79,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) submitApp(appContext) - + appId + } + + def run() { + val appId = runApp() monitorApplication(appId) System.exit(0) } @@ -372,7 +377,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val commands = List[String](javaCommand + " -server " + JAVA_OPTS + - " org.apache.spark.deploy.yarn.ApplicationMaster" + + " " + args.amClass + " --class " + args.userClass + " --jar " + args.userJar + userArgsToString(args) + diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 852dbd7dabf66..b9dbc3fb87a1f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -35,6 +35,7 @@ class ClientArguments(val args: Array[String]) { var numWorkers = 2 var amQueue = System.getProperty("QUEUE", "default") var amMemory: Int = 512 + var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" // TODO var inputFormatInfo: List[InputFormatInfo] = null @@ -62,18 +63,22 @@ class ClientArguments(val args: Array[String]) { userArgsBuffer += value args = tail - case ("--master-memory") :: MemoryParam(value) :: tail => - amMemory = value + case ("--master-class") :: value :: tail => + amClass = value args = tail - case ("--num-workers") :: IntParam(value) :: tail => - numWorkers = value + case ("--master-memory") :: MemoryParam(value) :: tail => + amMemory = value args = tail case ("--worker-memory") :: MemoryParam(value) :: tail => workerMemory = value args = tail + case ("--num-workers") :: IntParam(value) :: tail => + numWorkers = value + args = tail + case ("--worker-cores") :: IntParam(value) :: tail => workerCores = value args = tail @@ -119,19 +124,20 @@ class ClientArguments(val args: Array[String]) { System.err.println( "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required)\n" + - " --class CLASS_NAME Name of your application's main class (required)\n" + - " --args ARGS Arguments to be passed to your application's main class.\n" + - " Mutliple invocations are possible, each will be passed in order.\n" + - " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + - " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + - " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + - " --name NAME The name of your application (Default: Spark)\n" + - " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + - " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + - " --files files Comma separated list of files to be distributed with the job.\n" + - " --archives archives Comma separated list of archives to be distributed with the job." + " --jar JAR_PATH Path to your application's JAR file (required)\n" + + " --class CLASS_NAME Name of your application's main class (required)\n" + + " --args ARGS Arguments to be passed to your application's main class.\n" + + " Mutliple invocations are possible, each will be passed in order.\n" + + " --num-workers NUM Number of workers to start (Default: 2)\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + + " --master-class CLASS_NAME Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" + + " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + + " --name NAME The name of your application (Default: Spark)\n" + + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + + " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + + " --files files Comma separated list of files to be distributed with the job.\n" + + " --archives archives Comma separated list of archives to be distributed with the job." ) System.exit(exitCode) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala new file mode 100644 index 0000000000000..421a83c87afdf --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import java.net.Socket +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} +import akka.actor._ +import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import akka.remote.RemoteClientShutdown +import akka.actor.Terminated +import akka.remote.RemoteClientDisconnected +import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.SplitInfo + +class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging { + + def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + + private val rpc: YarnRPC = YarnRPC.create(conf) + private var resourceManager: AMRMProtocol = null + private var appAttemptId: ApplicationAttemptId = null + private var reporterThread: Thread = null + private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + + private var yarnAllocator: YarnAllocationHandler = null + private var driverClosed:Boolean = false + + val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1 + var actor: ActorRef = null + + // This actor just working as a monitor to watch on Driver Actor. + class MonitorActor(driverUrl: String) extends Actor { + + var driver: ActorRef = null + + override def preStart() { + logInfo("Listen to driver: " + driverUrl) + driver = context.actorFor(driverUrl) + driver ! "hello" + context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.watch(driver) // Doesn't work with remote actors, but useful for testing + } + + override def receive = { + case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + logInfo("Driver terminated or disconnected! Shutting down.") + driverClosed = true + } + } + + def run() { + + appAttemptId = getApplicationAttemptId() + resourceManager = registerWithResourceManager() + val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() + + // Compute number of threads for akka + val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() + + if (minimumMemory > 0) { + val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD + val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) + + if (numCore > 0) { + // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 + // TODO: Uncomment when hadoop is on a version which has this fixed. + // args.workerCores = numCore + } + } + + waitForSparkMaster() + + // Allocate all containers + allocateWorkers() + + // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout + // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. + + val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) + // must be <= timeoutInterval/ 2. + // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. + // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. + val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L)) + reporterThread = launchReporterThread(interval) + + // Wait for the reporter thread to Finish. + reporterThread.join() + + finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + actorSystem.shutdown() + + logInfo("Exited") + System.exit(0) + } + + private def getApplicationAttemptId(): ApplicationAttemptId = { + val envs = System.getenv() + val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) + val containerId = ConverterUtils.toContainerId(containerIdString) + val appAttemptId = containerId.getApplicationAttemptId() + logInfo("ApplicationAttemptId: " + appAttemptId) + return appAttemptId + } + + private def registerWithResourceManager(): AMRMProtocol = { + val rmAddress = NetUtils.createSocketAddr(yarnConf.get( + YarnConfiguration.RM_SCHEDULER_ADDRESS, + YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) + logInfo("Connecting to ResourceManager at " + rmAddress) + return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] + } + + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { + logInfo("Registering the ApplicationMaster") + val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) + .asInstanceOf[RegisterApplicationMasterRequest] + appMasterRequest.setApplicationAttemptId(appAttemptId) + // Setting this to master host,port - so that the ApplicationReport at client has some sensible info. + // Users can then monitor stderr/stdout on that node if required. + appMasterRequest.setHost(Utils.localHostName()) + appMasterRequest.setRpcPort(0) + // What do we provide here ? Might make sense to expose something sensible later ? + appMasterRequest.setTrackingUrl("") + return resourceManager.registerApplicationMaster(appMasterRequest) + } + + private def waitForSparkMaster() { + logInfo("Waiting for spark driver to be reachable.") + var driverUp = false + val hostport = args.userArgs(0) + val (driverHost, driverPort) = Utils.parseHostPort(hostport) + while(!driverUp) { + try { + val socket = new Socket(driverHost, driverPort) + socket.close() + logInfo("Master now available: " + driverHost + ":" + driverPort) + driverUp = true + } catch { + case e: Exception => + logError("Failed to connect to driver at " + driverHost + ":" + driverPort) + Thread.sleep(100) + } + } + System.setProperty("spark.driver.host", driverHost) + System.setProperty("spark.driver.port", driverPort.toString) + + val driverUrl = "akka://spark@%s:%s/user/%s".format( + driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + + actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") + } + + + private def allocateWorkers() { + + // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. + val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = scala.collection.immutable.Map() + + yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, preferredNodeLocationData) + + logInfo("Allocating " + args.numWorkers + " workers.") + // Wait until all containers have finished + // TODO: This is a bit ugly. Can we make it nicer? + // TODO: Handle container failure + while(yarnAllocator.getNumWorkersRunning < args.numWorkers) { + yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + Thread.sleep(100) + } + + logInfo("All workers have launched.") + + } + + // TODO: We might want to extend this to allocate more containers in case they die ! + private def launchReporterThread(_sleepTime: Long): Thread = { + val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime + + val t = new Thread { + override def run() { + while (!driverClosed) { + val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning + if (missingWorkerCount > 0) { + logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") + yarnAllocator.allocateContainers(missingWorkerCount) + } + else sendProgress() + Thread.sleep(sleepTime) + } + } + } + // setting to daemon status, though this is usually not a good idea. + t.setDaemon(true) + t.start() + logInfo("Started progress reporter thread - sleep time : " + sleepTime) + return t + } + + private def sendProgress() { + logDebug("Sending progress") + // simulated with an allocate request with no nodes requested ... + yarnAllocator.allocateContainers(0) + } + + def finishApplicationMaster(status: FinalApplicationStatus) { + + logInfo("finish ApplicationMaster with " + status) + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + resourceManager.finishApplicationMaster(finishReq) + } + +} + + +object WorkerLauncher { + def main(argStrings: Array[String]) { + val args = new ApplicationMasterArguments(argStrings) + new WorkerLauncher(args).run() + } +} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala new file mode 100644 index 0000000000000..63a0449e5a073 --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.scheduler.cluster + +import org.apache.spark._ +import org.apache.hadoop.conf.Configuration +import org.apache.spark.deploy.yarn.YarnAllocationHandler +import org.apache.spark.util.Utils + +/** + * + * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM. + */ +private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) { + + def this(sc: SparkContext) = this(sc, new Configuration()) + + // By default, rack is unknown + override def getRackForHost(hostPort: String): Option[String] = { + val host = Utils.parseHostPort(hostPort)._1 + val retval = YarnAllocationHandler.lookupRack(conf, host) + if (retval != null) Some(retval) else None + } + + override def postStartHook() { + + // The yarn application is running, but the worker might not yet ready + // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt + Thread.sleep(2000L) + logInfo("YarnClientClusterScheduler.postStartHook done") + } +} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala new file mode 100644 index 0000000000000..b206780c7806e --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.scheduler.cluster + +import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} +import org.apache.spark.{SparkException, Logging, SparkContext} +import org.apache.spark.deploy.yarn.{Client, ClientArguments} + +private[spark] class YarnClientSchedulerBackend( + scheduler: ClusterScheduler, + sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + with Logging { + + var client: Client = null + var appId: ApplicationId = null + + override def start() { + super.start() + + val defalutWorkerCores = "2" + val defalutWorkerMemory = "512m" + val defaultWorkerNumber = "1" + + val userJar = System.getenv("SPARK_YARN_APP_JAR") + var workerCores = System.getenv("SPARK_WORKER_CORES") + var workerMemory = System.getenv("SPARK_WORKER_MEMORY") + var workerNumber = System.getenv("SPARK_WORKER_INSTANCES") + + if (userJar == null) + throw new SparkException("env SPARK_YARN_APP_JAR is not set") + + if (workerCores == null) + workerCores = defalutWorkerCores + if (workerMemory == null) + workerMemory = defalutWorkerMemory + if (workerNumber == null) + workerNumber = defaultWorkerNumber + + val driverHost = System.getProperty("spark.driver.host") + val driverPort = System.getProperty("spark.driver.port") + val hostport = driverHost + ":" + driverPort + + val argsArray = Array[String]( + "--class", "notused", + "--jar", userJar, + "--args", hostport, + "--worker-memory", workerMemory, + "--worker-cores", workerCores, + "--num-workers", workerNumber, + "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" + ) + + val args = new ClientArguments(argsArray) + client = new Client(args) + appId = client.runApp() + waitForApp() + } + + def waitForApp() { + + // TODO : need a better way to find out whether the workers are ready or not + // maybe by resource usage report? + while(true) { + val report = client.getApplicationReport(appId) + + logInfo("Application report from ASM: \n" + + "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + + "\t appStartTime: " + report.getStartTime() + "\n" + + "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + ) + + // Ready to go, or already gone. + val state = report.getYarnApplicationState() + if (state == YarnApplicationState.RUNNING) { + return + } else if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + throw new SparkException("Yarn application already ended," + + "might be killed or not able to launch application master.") + } + + Thread.sleep(1000) + } + } + + override def stop() { + super.stop() + client.stop() + logInfo("Stoped") + } + +} From 2d3eae244bc1b5e6a22798cd307fba3026053edc Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 4 Dec 2013 23:33:04 -0800 Subject: [PATCH 36/90] Merge pull request #199 from harveyfeng/yarn-2.2 Hadoop 2.2 migration Includes support for the YARN API stabilized in the Hadoop 2.2 release, and a few style patches. Short description for each set of commits: a98f5a0 - "Misc style changes in the 'yarn' package" a67ebf4 - "A few more style fixes in the 'yarn' package" Both of these are some minor style changes, such as fixing lines over 100 chars, to the existing YARN code. ab8652f - "Add a 'new-yarn' directory ... " Copies everything from `SPARK_HOME/yarn` to `SPARK_HOME/new-yarn`. No actual code changes here. 4f1c3fa - "Hadoop 2.2 YARN API migration ..." API patches to code in the `SPARK_HOME/new-yarn` directory. There are a few more small style changes mixed in, too. Based on @colorant's Hadoop 2.2 support for the scala-2.10 branch in #141. a1a1c62 - "Add optional Hadoop 2.2 settings in sbt build ... " If Spark should be built against Hadoop 2.2, then: a) the `org.apache.spark.deploy.yarn` package will be compiled from the `new-yarn` directory. b) Protobuf v2.5 will be used as a Spark dependency, since Hadoop 2.2 depends on it. Also, Spark will be built against a version of Akka v2.0.5 that's built against Protobuf 2.5, named `akka-2.0.5-protobuf-2.5`. The patched Akka is here: https://github.com/harveyfeng/akka/tree/2.0.5-protobuf-2.5, and was published to local Ivy during testing. There's also a new boolean environment variable, `SPARK_IS_NEW_HADOOP`, that users can manually set if their `SPARK_HADOOP_VERSION` specification does not start with `2.2`, which is how the build file tries to detect a 2.2 version. Not sure if this is necessary or done in the best way, though... (cherry picked from commit 72b696156c8662cae2cef4b943520b4be86148ea) Conflicts: project/SparkBuild.scala streaming/pom.xml --- core/pom.xml | 10 +- .../scala/org/apache/spark/SparkContext.scala | 2 +- new-yarn/pom.xml | 161 ++++ .../spark/deploy/yarn/ApplicationMaster.scala | 446 ++++++++++++ .../yarn/ApplicationMasterArguments.scala | 94 +++ .../org/apache/spark/deploy/yarn/Client.scala | 519 +++++++++++++ .../spark/deploy/yarn/ClientArguments.scala | 148 ++++ .../yarn/ClientDistributedCacheManager.scala | 228 ++++++ .../spark/deploy/yarn/WorkerLauncher.scala | 223 ++++++ .../spark/deploy/yarn/WorkerRunnable.scala | 209 ++++++ .../deploy/yarn/YarnAllocationHandler.scala | 687 ++++++++++++++++++ .../deploy/yarn/YarnSparkHadoopUtil.scala | 43 ++ .../cluster/YarnClientClusterScheduler.scala | 47 ++ .../cluster/YarnClientSchedulerBackend.scala | 109 +++ .../cluster/YarnClusterScheduler.scala | 55 ++ .../ClientDistributedCacheManagerSuite.scala | 220 ++++++ pom.xml | 61 +- project/SparkBuild.scala | 35 +- streaming/pom.xml | 3 +- .../spark/deploy/yarn/ApplicationMaster.scala | 172 +++-- .../org/apache/spark/deploy/yarn/Client.scala | 151 ++-- .../spark/deploy/yarn/WorkerRunnable.scala | 85 ++- .../deploy/yarn/YarnAllocationHandler.scala | 346 +++++---- 23 files changed, 3717 insertions(+), 337 deletions(-) create mode 100644 new-yarn/pom.xml create mode 100644 new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala create mode 100644 new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala create mode 100644 new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala diff --git a/core/pom.xml b/core/pom.xml index 5e9136fef156c..1264bc4933cfe 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -95,15 +95,15 @@ 0.3.1 - com.typesafe.akka + ${akka.group} akka-actor - com.typesafe.akka + ${akka.group} akka-remote - com.typesafe.akka + ${akka.group} akka-slf4j @@ -126,10 +126,6 @@ colt colt - - com.github.scala-incubator.io - scala-io-file_2.9.2 - org.apache.mesos mesos diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 82be5581d6b11..eb5bb17539fb6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -81,7 +81,7 @@ class SparkContext( val sparkHome: String = null, val jars: Seq[String] = Nil, val environment: Map[String, String] = Map(), - // This is used only by yarn for now, but should be relevant to other cluster types (mesos, etc) + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set // of data-local splits on host val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml new file mode 100644 index 0000000000000..8a065c6d7d1d7 --- /dev/null +++ b/new-yarn/pom.xml @@ -0,0 +1,161 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent + 0.9.0-incubating-SNAPSHOT + ../pom.xml + + + org.apache.spark + spark-yarn_2.9.3 + jar + Spark Project YARN Support + http://spark.incubator.apache.org/ + + + + org.apache.spark + spark-core_2.9.3 + ${project.version} + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-client + + + org.apache.hadoop + hadoop-client + ${yarn.version} + + + org.apache.avro + avro + + + org.apache.avro + avro-ipc + + + org.scalatest + scalatest_2.9.3 + test + + + org.mockito + mockito-all + test + + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + + + org.scalatest + scalatest-maven-plugin + + + ${basedir}/.. + 1 + ${spark.classpath} + + + + + + diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala new file mode 100644 index 0000000000000..eeeca3ea8a33e --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -0,0 +1,446 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import java.io.IOException +import java.net.Socket +import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.util.ShutdownHookManager +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} + +import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.util.Utils + + +class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { + + def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + + private var rpc: YarnRPC = YarnRPC.create(conf) + private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + private var appAttemptId: ApplicationAttemptId = _ + private var userThread: Thread = _ + private val fs = FileSystem.get(yarnConf) + + private var yarnAllocator: YarnAllocationHandler = _ + private var isFinished: Boolean = false + private var uiAddress: String = _ + private val maxAppAttempts: Int = conf.getInt( + YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) + private var isLastAMRetry: Boolean = true + private var amClient: AMRMClient[ContainerRequest] = _ + + // Default to numWorkers * 2, with minimum of 3 + private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", + math.max(args.numWorkers * 2, 3).toString()).toInt + + def run() { + // Setup the directories so things go to YARN approved directories rather + // than user specified and /tmp. + System.setProperty("spark.local.dir", getLocalDirs()) + + // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using. + ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) + + appAttemptId = getApplicationAttemptId() + isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts + amClient = AMRMClient.createAMRMClient() + amClient.init(yarnConf) + amClient.start() + + // Workaround until hadoop moves to something which has + // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line) + // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) + + ApplicationMaster.register(this) + + // Start the user's JAR + userThread = startUserClass() + + // This a bit hacky, but we need to wait until the spark.driver.port property has + // been set by the Thread executing the user class. + waitForSparkMaster() + + waitForSparkContextInitialized() + + // Do this after Spark master is up and SparkContext is created so that we can register UI Url. + val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() + + // Allocate all containers + allocateWorkers() + + // Wait for the user class to Finish + userThread.join() + + System.exit(0) + } + + /** Get the Yarn approved local directories. */ + private def getLocalDirs(): String = { + // Hadoop 0.23 and 2.x have different Environment variable names for the + // local dirs, so lets check both. We assume one of the 2 is set. + // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X + val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) + .getOrElse(Option(System.getenv("LOCAL_DIRS")) + .getOrElse("")) + + if (localDirs.isEmpty()) { + throw new Exception("Yarn Local dirs can't be empty") + } + localDirs + } + + private def getApplicationAttemptId(): ApplicationAttemptId = { + val envs = System.getenv() + val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name()) + val containerId = ConverterUtils.toContainerId(containerIdString) + val appAttemptId = containerId.getApplicationAttemptId() + logInfo("ApplicationAttemptId: " + appAttemptId) + appAttemptId + } + + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { + logInfo("Registering the ApplicationMaster") + amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + } + + private def waitForSparkMaster() { + logInfo("Waiting for Spark driver to be reachable.") + var driverUp = false + var tries = 0 + val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt + while (!driverUp && tries < numTries) { + val driverHost = System.getProperty("spark.driver.host") + val driverPort = System.getProperty("spark.driver.port") + try { + val socket = new Socket(driverHost, driverPort.toInt) + socket.close() + logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) + driverUp = true + } catch { + case e: Exception => { + logWarning("Failed to connect to driver at %s:%s, retrying ...". + format(driverHost, driverPort)) + Thread.sleep(100) + tries = tries + 1 + } + } + } + } + + private def startUserClass(): Thread = { + logInfo("Starting the user JAR in a separate Thread") + val mainMethod = Class.forName( + args.userClass, + false /* initialize */, + Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) + val t = new Thread { + override def run() { + var successed = false + try { + // Copy + var mainArgs: Array[String] = new Array[String](args.userArgs.size) + args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size) + mainMethod.invoke(null, mainArgs) + // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR + // userThread will stop here unless it has uncaught exception thrown out + // It need shutdown hook to set SUCCEEDED + successed = true + } finally { + logDebug("finishing main") + isLastAMRetry = true + if (successed) { + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + } else { + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) + } + } + } + } + t.start() + t + } + + // This need to happen before allocateWorkers() + private def waitForSparkContextInitialized() { + logInfo("Waiting for Spark context initialization") + try { + var sparkContext: SparkContext = null + ApplicationMaster.sparkContextRef.synchronized { + var numTries = 0 + val waitTime = 10000L + val maxNumTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt + while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { + logInfo("Waiting for Spark context initialization ... " + numTries) + numTries = numTries + 1 + ApplicationMaster.sparkContextRef.wait(waitTime) + } + sparkContext = ApplicationMaster.sparkContextRef.get() + assert(sparkContext != null || numTries >= maxNumTries) + + if (sparkContext != null) { + uiAddress = sparkContext.ui.appUIAddress + this.yarnAllocator = YarnAllocationHandler.newAllocator( + yarnConf, + amClient, + appAttemptId, + args, + sparkContext.preferredNodeLocationData) + } else { + logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d". + format(numTries * waitTime, maxNumTries)) + this.yarnAllocator = YarnAllocationHandler.newAllocator( + yarnConf, + amClient, + appAttemptId, + args) + } + } + } finally { + // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT : + // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks. + ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) + } + } + + private def allocateWorkers() { + try { + logInfo("Allocating " + args.numWorkers + " workers.") + // Wait until all containers have finished + // TODO: This is a bit ugly. Can we make it nicer? + // TODO: Handle container failure + yarnAllocator.addResourceRequests(args.numWorkers) + // Exits the loop if the user thread exits. + while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) { + if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of worker failures reached") + } + yarnAllocator.allocateResources() + ApplicationMaster.incrementAllocatorLoop(1) + Thread.sleep(100) + } + } finally { + // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT, + // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. + ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) + } + logInfo("All workers have launched.") + + // Launch a progress reporter thread, else the app will get killed after expiration + // (def: 10mins) timeout. + if (userThread.isAlive) { + // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses. + val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) + + // we want to be reasonably responsive without causing too many requests to RM. + val schedulerInterval = + System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + + // must be <= timeoutInterval / 2. + val interval = math.min(timeoutInterval / 2, schedulerInterval) + + launchReporterThread(interval) + } + } + + private def launchReporterThread(_sleepTime: Long): Thread = { + val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime + + val t = new Thread { + override def run() { + while (userThread.isAlive) { + if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of worker failures reached") + } + val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - + yarnAllocator.getNumPendingAllocate + if (missingWorkerCount > 0) { + logInfo("Allocating %d containers to make up for (potentially) lost containers". + format(missingWorkerCount)) + yarnAllocator.addResourceRequests(missingWorkerCount) + } + sendProgress() + Thread.sleep(sleepTime) + } + } + } + // Setting to daemon status, though this is usually not a good idea. + t.setDaemon(true) + t.start() + logInfo("Started progress reporter thread - sleep time : " + sleepTime) + t + } + + private def sendProgress() { + logDebug("Sending progress") + // Simulated with an allocate request with no nodes requested. + yarnAllocator.allocateResources() + } + + /* + def printContainers(containers: List[Container]) = { + for (container <- containers) { + logInfo("Launching shell command on a new container." + + ", containerId=" + container.getId() + + ", containerNode=" + container.getNodeId().getHost() + + ":" + container.getNodeId().getPort() + + ", containerNodeURI=" + container.getNodeHttpAddress() + + ", containerState" + container.getState() + + ", containerResourceMemory" + + container.getResource().getMemory()) + } + } + */ + + def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") { + synchronized { + if (isFinished) { + return + } + isFinished = true + } + + logInfo("finishApplicationMaster with " + status) + // Set tracking URL to empty since we don't have a history server. + amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */) + } + + /** + * Clean up the staging directory. + */ + private def cleanupStagingDir() { + var stagingDirPath: Path = null + try { + val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean + if (!preserveFiles) { + stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) + if (stagingDirPath == null) { + logError("Staging directory is null") + return + } + logInfo("Deleting staging directory " + stagingDirPath) + fs.delete(stagingDirPath, true) + } + } catch { + case ioe: IOException => + logError("Failed to cleanup staging dir " + stagingDirPath, ioe) + } + } + + // The shutdown hook that runs when a signal is received AND during normal close of the JVM. + class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable { + + def run() { + logInfo("AppMaster received a signal.") + // we need to clean up staging dir before HDFS is shut down + // make sure we don't delete it until this is the last AM + if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir() + } + } +} + +object ApplicationMaster { + // Number of times to wait for the allocator loop to complete. + // Each loop iteration waits for 100ms, so maximum of 3 seconds. + // This is to ensure that we have reasonable number of containers before we start + // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be + // optimal as more containers are available. Might need to handle this better. + private val ALLOCATOR_LOOP_WAIT_COUNT = 30 + + private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() + + val sparkContextRef: AtomicReference[SparkContext] = + new AtomicReference[SparkContext](null /* initialValue */) + + val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0) + + def incrementAllocatorLoop(by: Int) { + val count = yarnAllocatorLoop.getAndAdd(by) + if (count >= ALLOCATOR_LOOP_WAIT_COUNT) { + yarnAllocatorLoop.synchronized { + // to wake threads off wait ... + yarnAllocatorLoop.notifyAll() + } + } + } + + def register(master: ApplicationMaster) { + applicationMasters.add(master) + } + + // TODO(harvey): See whether this should be discarded - it isn't used anywhere atm... + def sparkContextInitialized(sc: SparkContext): Boolean = { + var modified = false + sparkContextRef.synchronized { + modified = sparkContextRef.compareAndSet(null, sc) + sparkContextRef.notifyAll() + } + + // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do + // System.exit. + // Should not really have to do this, but it helps YARN to evict resources earlier. + // Not to mention, prevent the Client from declaring failure even though we exited properly. + // Note that this will unfortunately not properly clean up the staging files because it gets + // called too late, after the filesystem is already shutdown. + if (modified) { + Runtime.getRuntime().addShutdownHook(new Thread with Logging { + // This is not only logs, but also ensures that log system is initialized for this instance + // when we are actually 'run'-ing. + logInfo("Adding shutdown hook for context " + sc) + override def run() { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() + // Best case ... + for (master <- applicationMasters) { + master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + } + } + } ) + } + + // Wait for initialization to complete and atleast 'some' nodes can get allocated. + yarnAllocatorLoop.synchronized { + while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) { + yarnAllocatorLoop.wait(1000L) + } + } + modified + } + + def main(argStrings: Array[String]) { + val args = new ApplicationMasterArguments(argStrings) + new ApplicationMaster(args).run() + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala new file mode 100644 index 0000000000000..f76a5ddd39e90 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import org.apache.spark.util.IntParam +import collection.mutable.ArrayBuffer + +class ApplicationMasterArguments(val args: Array[String]) { + var userJar: String = null + var userClass: String = null + var userArgs: Seq[String] = Seq[String]() + var workerMemory = 1024 + var workerCores = 1 + var numWorkers = 2 + + parseArgs(args.toList) + + private def parseArgs(inputArgs: List[String]): Unit = { + val userArgsBuffer = new ArrayBuffer[String]() + + var args = inputArgs + + while (! args.isEmpty) { + + args match { + case ("--jar") :: value :: tail => + userJar = value + args = tail + + case ("--class") :: value :: tail => + userClass = value + args = tail + + case ("--args") :: value :: tail => + userArgsBuffer += value + args = tail + + case ("--num-workers") :: IntParam(value) :: tail => + numWorkers = value + args = tail + + case ("--worker-memory") :: IntParam(value) :: tail => + workerMemory = value + args = tail + + case ("--worker-cores") :: IntParam(value) :: tail => + workerCores = value + args = tail + + case Nil => + if (userJar == null || userClass == null) { + printUsageAndExit(1) + } + + case _ => + printUsageAndExit(1, args) + } + } + + userArgs = userArgsBuffer.readOnly + } + + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + if (unknownParam != null) { + System.err.println("Unknown/unsupported param " + unknownParam) + } + System.err.println( + "Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options] \n" + + "Options:\n" + + " --jar JAR_PATH Path to your application's JAR file (required)\n" + + " --class CLASS_NAME Name of your application's main class (required)\n" + + " --args ARGS Arguments to be passed to your application's main class.\n" + + " Mutliple invocations are possible, each will be passed in order.\n" + + " --num-workers NUM Number of workers to start (Default: 2)\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n") + System.exit(exitCode) + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala new file mode 100644 index 0000000000000..94678815e806a --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -0,0 +1,519 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import java.net.{InetAddress, UnknownHostException, URI} +import java.nio.ByteBuffer + +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap +import scala.collection.mutable.Map + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil} +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.mapred.Master +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{Apps, Records} + +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil + + +/** + * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The + * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, + * which will launch a Spark master process and negotiate resources throughout its duration. + */ +class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { + + var rpc: YarnRPC = YarnRPC.create(conf) + val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + val credentials = UserGroupInformation.getCurrentUser().getCredentials() + private val SPARK_STAGING: String = ".sparkStaging" + private val distCacheMgr = new ClientDistributedCacheManager() + + // Staging directory is private! -> rwx-------- + val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short) + // App files are world-wide readable and owner writable -> rw-r--r-- + val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short) + + def this(args: ClientArguments) = this(new Configuration(), args) + + def runApp(): ApplicationId = { + validateArgs() + // Initialize and start the client service. + init(yarnConf) + start() + + // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers). + logClusterResourceDetails() + + // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM) + // interface). + + // Get a new client application. + val newApp = super.createApplication() + val newAppResponse = newApp.getNewApplicationResponse() + val appId = newAppResponse.getApplicationId() + + verifyClusterResources(newAppResponse) + + // Set up resource and environment variables. + val appStagingDir = getAppStagingDir(appId) + val localResources = prepareLocalResources(appStagingDir) + val launchEnv = setupLaunchEnv(localResources, appStagingDir) + val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv) + + // Set up an application submission context. + val appContext = newApp.getApplicationSubmissionContext() + appContext.setApplicationName(args.appName) + appContext.setQueue(args.amQueue) + appContext.setAMContainerSpec(amContainer) + + // Memory for the ApplicationMaster. + val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] + memoryResource.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + appContext.setResource(memoryResource) + + // Finally, submit and monitor the application. + submitApp(appContext) + appId + } + + def run() { + val appId = runApp() + monitorApplication(appId) + System.exit(0) + } + + // TODO(harvey): This could just go in ClientArguments. + def validateArgs() = { + Map( + (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", + (args.userJar == null) -> "Error: You must specify a user jar!", + (args.userClass == null) -> "Error: You must specify a user class!", + (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!", + (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" + + "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), + (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" + + "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString) + ).foreach { case(cond, errStr) => + if (cond) { + logError(errStr) + args.printUsageAndExit(1) + } + } + } + + def getAppStagingDir(appId: ApplicationId): String = { + SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR + } + + def logClusterResourceDetails() { + val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics + logInfo("Got Cluster metric info from ApplicationsManager (ASM), number of NodeManagers: " + + clusterMetrics.getNumNodeManagers) + + val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) + logInfo("""Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s, + queueApplicationCount = %s, queueChildQueueCount = %s""".format( + queueInfo.getQueueName, + queueInfo.getCurrentCapacity, + queueInfo.getMaximumCapacity, + queueInfo.getApplications.size, + queueInfo.getChildQueues.size)) + } + + def verifyClusterResources(app: GetNewApplicationResponse) = { + val maxMem = app.getMaximumResourceCapability().getMemory() + logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) + + // If we have requested more then the clusters max for a single resource then exit. + if (args.workerMemory > maxMem) { + logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.". + format(args.workerMemory, maxMem)) + System.exit(1) + } + val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD + if (amMem > maxMem) { + logError("Required AM memory (%d) is above the max threshold (%d) of this cluster". + format(args.amMemory, maxMem)) + System.exit(1) + } + + // We could add checks to make sure the entire cluster has enough resources but that involves + // getting all the node reports and computing ourselves. + } + + /** See if two file systems are the same or not. */ + private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { + val srcUri = srcFs.getUri() + val dstUri = destFs.getUri() + if (srcUri.getScheme() == null) { + return false + } + if (!srcUri.getScheme().equals(dstUri.getScheme())) { + return false + } + var srcHost = srcUri.getHost() + var dstHost = dstUri.getHost() + if ((srcHost != null) && (dstHost != null)) { + try { + srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() + dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() + } catch { + case e: UnknownHostException => + return false + } + if (!srcHost.equals(dstHost)) { + return false + } + } else if (srcHost == null && dstHost != null) { + return false + } else if (srcHost != null && dstHost == null) { + return false + } + //check for ports + if (srcUri.getPort() != dstUri.getPort()) { + return false + } + return true + } + + /** Copy the file into HDFS if needed. */ + private def copyRemoteFile( + dstDir: Path, + originalPath: Path, + replication: Short, + setPerms: Boolean = false): Path = { + val fs = FileSystem.get(conf) + val remoteFs = originalPath.getFileSystem(conf) + var newPath = originalPath + if (! compareFs(remoteFs, fs)) { + newPath = new Path(dstDir, originalPath.getName()) + logInfo("Uploading " + originalPath + " to " + newPath) + FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) + fs.setReplication(newPath, replication) + if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) + } + // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific + // version shows the specific version in the distributed cache configuration + val qualPath = fs.makeQualified(newPath) + val fc = FileContext.getFileContext(qualPath.toUri(), conf) + val destPath = fc.resolvePath(qualPath) + destPath + } + + def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { + logInfo("Preparing Local resources") + // Upload Spark and the application JAR to the remote file system if necessary. Add them as + // local resources to the application master. + val fs = FileSystem.get(conf) + + val delegTokenRenewer = Master.getMasterPrincipal(conf) + if (UserGroupInformation.isSecurityEnabled()) { + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + logError("Can't get Master Kerberos principal for use as renewer") + System.exit(1) + } + } + val dst = new Path(fs.getHomeDirectory(), appStagingDir) + val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort + + if (UserGroupInformation.isSecurityEnabled()) { + val dstFs = dst.getFileSystem(conf) + dstFs.addDelegationTokens(delegTokenRenewer, credentials) + } + + val localResources = HashMap[String, LocalResource]() + FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) + + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + + Map( + Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar, + Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF") + ).foreach { case(destName, _localPath) => + val localPath: String = if (_localPath != null) _localPath.trim() else "" + if (! localPath.isEmpty()) { + var localURI = new URI(localPath) + // If not specified assume these are in the local filesystem to keep behavior like Hadoop + if (localURI.getScheme() == null) { + localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString) + } + val setPermissions = if (destName.equals(Client.APP_JAR)) true else false + val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions) + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + destName, statCache) + } + } + + // Handle jars local to the ApplicationMaster. + if ((args.addJars != null) && (!args.addJars.isEmpty())){ + args.addJars.split(',').foreach { case file: String => + val localURI = new URI(file.trim()) + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyRemoteFile(dst, localPath, replication) + // Only add the resource to the Spark ApplicationMaster. + val appMasterOnly = true + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + linkname, statCache, appMasterOnly) + } + } + + // Handle any distributed cache files + if ((args.files != null) && (!args.files.isEmpty())){ + args.files.split(',').foreach { case file: String => + val localURI = new URI(file.trim()) + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyRemoteFile(dst, localPath, replication) + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + linkname, statCache) + } + } + + // Handle any distributed cache archives + if ((args.archives != null) && (!args.archives.isEmpty())) { + args.archives.split(',').foreach { case file:String => + val localURI = new URI(file.trim()) + val localPath = new Path(localURI) + val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) + val destPath = copyRemoteFile(dst, localPath, replication) + distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, + linkname, statCache) + } + } + + UserGroupInformation.getCurrentUser().addCredentials(credentials) + localResources + } + + def setupLaunchEnv( + localResources: HashMap[String, LocalResource], + stagingDir: String): HashMap[String, String] = { + logInfo("Setting up the launch environment") + val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null) + + val env = new HashMap[String, String]() + + Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env) + env("SPARK_YARN_MODE") = "true" + env("SPARK_YARN_STAGING_DIR") = stagingDir + + // Set the environment variables to be passed on to the Workers. + distCacheMgr.setDistFilesEnv(env) + distCacheMgr.setDistArchivesEnv(env) + + // Allow users to specify some environment variables. + Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) + + // Add each SPARK_* key to the environment. + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + + env + } + + def userArgsToString(clientArgs: ClientArguments): String = { + val prefix = " --args " + val args = clientArgs.userArgs + val retval = new StringBuilder() + for (arg <- args){ + retval.append(prefix).append(" '").append(arg).append("' ") + } + retval.toString + } + + def createContainerLaunchContext( + newApp: GetNewApplicationResponse, + localResources: HashMap[String, LocalResource], + env: HashMap[String, String]): ContainerLaunchContext = { + logInfo("Setting up container launch context") + val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) + amContainer.setLocalResources(localResources) + amContainer.setEnvironment(env) + + // TODO: Need a replacement for the following code to fix -Xmx? + // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() + // var amMemory = ((args.amMemory / minResMemory) * minResMemory) + + // ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - + // YarnAllocationHandler.MEMORY_OVERHEAD) + + // Extra options for the JVM + var JAVA_OPTS = "" + + // Add Xmx for AM memory + JAVA_OPTS += "-Xmx" + args.amMemory + "m" + + val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir + + // TODO: Remove once cpuset version is pushed out. + // The context is, default gc for server class machines ends up using all cores to do gc - + // hence if there are multiple containers in same node, Spark GC affects all other containers' + // performance (which can be that of other Spark containers) + // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in + // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset + // of cores on a node. + val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && + java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC")) + if (useConcurrentAndIncrementalGC) { + // In our expts, using (default) throughput collector has severe perf ramifications in + // multi-tenant machines + JAVA_OPTS += " -XX:+UseConcMarkSweepGC " + JAVA_OPTS += " -XX:+CMSIncrementalMode " + JAVA_OPTS += " -XX:+CMSIncrementalPacing " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + } + + if (env.isDefinedAt("SPARK_JAVA_OPTS")) { + JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") + } + + // Command for the ApplicationMaster + var javaCommand = "java" + val javaHome = System.getenv("JAVA_HOME") + if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { + javaCommand = Environment.JAVA_HOME.$() + "/bin/java" + } + + val commands = List[String]( + javaCommand + + " -server " + + JAVA_OPTS + + " " + args.amClass + + " --class " + args.userClass + + " --jar " + args.userJar + + userArgsToString(args) + + " --worker-memory " + args.workerMemory + + " --worker-cores " + args.workerCores + + " --num-workers " + args.numWorkers + + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + logInfo("Command for starting the Spark ApplicationMaster: " + commands(0)) + amContainer.setCommands(commands) + + // Setup security tokens. + val dob = new DataOutputBuffer() + credentials.writeTokenStorageToStream(dob) + amContainer.setTokens(ByteBuffer.wrap(dob.getData())) + + amContainer + } + + def submitApp(appContext: ApplicationSubmissionContext) = { + // Submit the application to the applications manager. + logInfo("Submitting application to ASM") + super.submitApplication(appContext) + } + + def monitorApplication(appId: ApplicationId): Boolean = { + while (true) { + Thread.sleep(1000) + val report = super.getApplicationReport(appId) + + logInfo("Application report from ASM: \n" + + "\t application identifier: " + appId.toString() + "\n" + + "\t appId: " + appId.getId() + "\n" + + "\t clientToAMToken: " + report.getClientToAMToken() + "\n" + + "\t appDiagnostics: " + report.getDiagnostics() + "\n" + + "\t appMasterHost: " + report.getHost() + "\n" + + "\t appQueue: " + report.getQueue() + "\n" + + "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + + "\t appStartTime: " + report.getStartTime() + "\n" + + "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + + "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" + + "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + + "\t appUser: " + report.getUser() + ) + + val state = report.getYarnApplicationState() + val dsStatus = report.getFinalApplicationStatus() + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + return true + } + } + true + } +} + +object Client { + val SPARK_JAR: String = "spark.jar" + val APP_JAR: String = "app.jar" + val LOG4J_PROP: String = "log4j.properties" + + def main(argStrings: Array[String]) { + // Set an env variable indicating we are running in YARN mode. + // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - + // see Client#setupLaunchEnv(). + System.setProperty("SPARK_YARN_MODE", "true") + + val args = new ClientArguments(argStrings) + + (new Client(args)).run() + } + + // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps + def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { + for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim) + } + } + + def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$()) + // If log4j present, ensure ours overrides all others + if (addLog4j) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + LOG4J_PROP) + } + // Normally the users app.jar is last in case conflicts with spark jars + val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") + .toBoolean + if (userClasspathFirst) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + APP_JAR) + } + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + SPARK_JAR) + Client.populateHadoopClasspath(conf, env) + + if (!userClasspathFirst) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + APP_JAR) + } + Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + + Path.SEPARATOR + "*") + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala new file mode 100644 index 0000000000000..9efb28a942672 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import scala.collection.mutable.{ArrayBuffer, HashMap} + +import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} +import org.apache.spark.util.IntParam +import org.apache.spark.util.MemoryParam + + +// TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! +class ClientArguments(val args: Array[String]) { + var addJars: String = null + var files: String = null + var archives: String = null + var userJar: String = null + var userClass: String = null + var userArgs: Seq[String] = Seq[String]() + var workerMemory = 1024 // MB + var workerCores = 1 + var numWorkers = 2 + var amQueue = System.getProperty("QUEUE", "default") + var amMemory: Int = 512 // MB + var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" + var appName: String = "Spark" + // TODO + var inputFormatInfo: List[InputFormatInfo] = null + // TODO(harvey) + var priority = 0 + + parseArgs(args.toList) + + private def parseArgs(inputArgs: List[String]): Unit = { + val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() + val inputFormatMap: HashMap[String, InputFormatInfo] = new HashMap[String, InputFormatInfo]() + + var args = inputArgs + + while (!args.isEmpty) { + args match { + case ("--jar") :: value :: tail => + userJar = value + args = tail + + case ("--class") :: value :: tail => + userClass = value + args = tail + + case ("--args") :: value :: tail => + userArgsBuffer += value + args = tail + + case ("--master-class") :: value :: tail => + amClass = value + args = tail + + case ("--master-memory") :: MemoryParam(value) :: tail => + amMemory = value + args = tail + + case ("--num-workers") :: IntParam(value) :: tail => + numWorkers = value + args = tail + + case ("--worker-memory") :: MemoryParam(value) :: tail => + workerMemory = value + args = tail + + case ("--worker-cores") :: IntParam(value) :: tail => + workerCores = value + args = tail + + case ("--queue") :: value :: tail => + amQueue = value + args = tail + + case ("--name") :: value :: tail => + appName = value + + case ("--addJars") :: value :: tail => + addJars = value + args = tail + + case ("--files") :: value :: tail => + files = value + args = tail + + case ("--archives") :: value :: tail => + archives = value + args = tail + + case Nil => + if (userJar == null || userClass == null) { + printUsageAndExit(1) + } + + case _ => + printUsageAndExit(1, args) + } + } + + userArgs = userArgsBuffer.readOnly + inputFormatInfo = inputFormatMap.values.toList + } + + + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + if (unknownParam != null) { + System.err.println("Unknown/unsupported param " + unknownParam) + } + System.err.println( + "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + + "Options:\n" + + " --jar JAR_PATH Path to your application's JAR file (required)\n" + + " --class CLASS_NAME Name of your application's main class (required)\n" + + " --args ARGS Arguments to be passed to your application's main class.\n" + + " Mutliple invocations are possible, each will be passed in order.\n" + + " --num-workers NUM Number of workers to start (Default: 2)\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + + " --master-class CLASS_NAME Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" + + " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + + " --name NAME The name of your application (Default: Spark)\n" + + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + + " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + + " --files files Comma separated list of files to be distributed with the job.\n" + + " --archives archives Comma separated list of archives to be distributed with the job." + ) + System.exit(exitCode) + } + +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala new file mode 100644 index 0000000000000..5f159b073f537 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import java.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.permission.FsAction +import org.apache.hadoop.yarn.api.records.LocalResource +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility +import org.apache.hadoop.yarn.api.records.LocalResourceType +import org.apache.hadoop.yarn.util.{Records, ConverterUtils} + +import org.apache.spark.Logging + +import scala.collection.mutable.HashMap +import scala.collection.mutable.LinkedHashMap +import scala.collection.mutable.Map + + +/** Client side methods to setup the Hadoop distributed cache */ +class ClientDistributedCacheManager() extends Logging { + private val distCacheFiles: Map[String, Tuple3[String, String, String]] = + LinkedHashMap[String, Tuple3[String, String, String]]() + private val distCacheArchives: Map[String, Tuple3[String, String, String]] = + LinkedHashMap[String, Tuple3[String, String, String]]() + + + /** + * Add a resource to the list of distributed cache resources. This list can + * be sent to the ApplicationMaster and possibly the workers so that it can + * be downloaded into the Hadoop distributed cache for use by this application. + * Adds the LocalResource to the localResources HashMap passed in and saves + * the stats of the resources to they can be sent to the workers and verified. + * + * @param fs FileSystem + * @param conf Configuration + * @param destPath path to the resource + * @param localResources localResource hashMap to insert the resource into + * @param resourceType LocalResourceType + * @param link link presented in the distributed cache to the destination + * @param statCache cache to store the file/directory stats + * @param appMasterOnly Whether to only add the resource to the app master + */ + def addResource( + fs: FileSystem, + conf: Configuration, + destPath: Path, + localResources: HashMap[String, LocalResource], + resourceType: LocalResourceType, + link: String, + statCache: Map[URI, FileStatus], + appMasterOnly: Boolean = false) = { + val destStatus = fs.getFileStatus(destPath) + val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + amJarRsrc.setType(resourceType) + val visibility = getVisibility(conf, destPath.toUri(), statCache) + amJarRsrc.setVisibility(visibility) + amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(destPath)) + amJarRsrc.setTimestamp(destStatus.getModificationTime()) + amJarRsrc.setSize(destStatus.getLen()) + if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name") + localResources(link) = amJarRsrc + + if (appMasterOnly == false) { + val uri = destPath.toUri() + val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link) + if (resourceType == LocalResourceType.FILE) { + distCacheFiles(pathURI.toString()) = (destStatus.getLen().toString(), + destStatus.getModificationTime().toString(), visibility.name()) + } else { + distCacheArchives(pathURI.toString()) = (destStatus.getLen().toString(), + destStatus.getModificationTime().toString(), visibility.name()) + } + } + } + + /** + * Adds the necessary cache file env variables to the env passed in + * @param env + */ + def setDistFilesEnv(env: Map[String, String]) = { + val (keys, tupleValues) = distCacheFiles.unzip + val (sizes, timeStamps, visibilities) = tupleValues.unzip3 + + if (keys.size > 0) { + env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = + timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = + sizes.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_FILES_VISIBILITIES") = + visibilities.reduceLeft[String] { (acc,n) => acc + "," + n } + } + } + + /** + * Adds the necessary cache archive env variables to the env passed in + * @param env + */ + def setDistArchivesEnv(env: Map[String, String]) = { + val (keys, tupleValues) = distCacheArchives.unzip + val (sizes, timeStamps, visibilities) = tupleValues.unzip3 + + if (keys.size > 0) { + env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = + timeStamps.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") = + sizes.reduceLeft[String] { (acc,n) => acc + "," + n } + env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") = + visibilities.reduceLeft[String] { (acc,n) => acc + "," + n } + } + } + + /** + * Returns the local resource visibility depending on the cache file permissions + * @param conf + * @param uri + * @param statCache + * @return LocalResourceVisibility + */ + def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): + LocalResourceVisibility = { + if (isPublic(conf, uri, statCache)) { + return LocalResourceVisibility.PUBLIC + } + return LocalResourceVisibility.PRIVATE + } + + /** + * Returns a boolean to denote whether a cache file is visible to all(public) + * or not + * @param conf + * @param uri + * @param statCache + * @return true if the path in the uri is visible to all, false otherwise + */ + def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { + val fs = FileSystem.get(uri, conf) + val current = new Path(uri.getPath()) + //the leaf level file should be readable by others + if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) { + return false + } + return ancestorsHaveExecutePermissions(fs, current.getParent(), statCache) + } + + /** + * Returns true if all ancestors of the specified path have the 'execute' + * permission set for all users (i.e. that other users can traverse + * the directory heirarchy to the given path) + * @param fs + * @param path + * @param statCache + * @return true if all ancestors have the 'execute' permission set for all users + */ + def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, + statCache: Map[URI, FileStatus]): Boolean = { + var current = path + while (current != null) { + //the subdirs in the path should have execute permissions for others + if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) { + return false + } + current = current.getParent() + } + return true + } + + /** + * Checks for a given path whether the Other permissions on it + * imply the permission in the passed FsAction + * @param fs + * @param path + * @param action + * @param statCache + * @return true if the path in the uri is visible to all, false otherwise + */ + def checkPermissionOfOther(fs: FileSystem, path: Path, + action: FsAction, statCache: Map[URI, FileStatus]): Boolean = { + val status = getFileStatus(fs, path.toUri(), statCache) + val perms = status.getPermission() + val otherAction = perms.getOtherAction() + if (otherAction.implies(action)) { + return true + } + return false + } + + /** + * Checks to see if the given uri exists in the cache, if it does it + * returns the existing FileStatus, otherwise it stats the uri, stores + * it in the cache, and returns the FileStatus. + * @param fs + * @param uri + * @param statCache + * @return FileStatus + */ + def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = { + val stat = statCache.get(uri) match { + case Some(existstat) => existstat + case None => + val newStat = fs.getFileStatus(new Path(uri)) + statCache.put(uri, newStat) + newStat + } + return stat + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala new file mode 100644 index 0000000000000..c38f33e212fbf --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -0,0 +1,223 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import java.net.Socket +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.{ConverterUtils, Records} +import akka.actor._ +import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import akka.actor.Terminated +import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.SplitInfo +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest + +class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) extends Logging { + + def this(args: ApplicationMasterArguments) = this(args, new Configuration()) + + private var appAttemptId: ApplicationAttemptId = _ + private var reporterThread: Thread = _ + private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + + private var yarnAllocator: YarnAllocationHandler = _ + private var driverClosed:Boolean = false + + private var amClient: AMRMClient[ContainerRequest] = _ + + val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0)._1 + var actor: ActorRef = _ + + // This actor just working as a monitor to watch on Driver Actor. + class MonitorActor(driverUrl: String) extends Actor { + + var driver: ActorRef = null + + override def preStart() { + logInfo("Listen to driver: " + driverUrl) + driver = context.actorFor(driverUrl) + context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.watch(driver) // Doesn't work with remote actors, but useful for testing + } + + override def receive = { + case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + logInfo("Driver terminated or disconnected! Shutting down.") + driverClosed = true + } + } + + def run() { + + amClient = AMRMClient.createAMRMClient() + amClient.init(yarnConf) + amClient.start() + + appAttemptId = getApplicationAttemptId() + registerApplicationMaster() + + waitForSparkMaster() + + // Allocate all containers + allocateWorkers() + + // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout + // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. + + val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) + // must be <= timeoutInterval/ 2. + // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM. + // so atleast 1 minute or timeoutInterval / 10 - whichever is higher. + val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L)) + reporterThread = launchReporterThread(interval) + + // Wait for the reporter thread to Finish. + reporterThread.join() + + finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + actorSystem.shutdown() + + logInfo("Exited") + System.exit(0) + } + + private def getApplicationAttemptId(): ApplicationAttemptId = { + val envs = System.getenv() + val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name()) + val containerId = ConverterUtils.toContainerId(containerIdString) + val appAttemptId = containerId.getApplicationAttemptId() + logInfo("ApplicationAttemptId: " + appAttemptId) + appAttemptId + } + + private def registerApplicationMaster(): RegisterApplicationMasterResponse = { + logInfo("Registering the ApplicationMaster") + // TODO:(Raymond) Find out Spark UI address and fill in here? + amClient.registerApplicationMaster(Utils.localHostName(), 0, "") + } + + private def waitForSparkMaster() { + logInfo("Waiting for Spark driver to be reachable.") + var driverUp = false + val hostport = args.userArgs(0) + val (driverHost, driverPort) = Utils.parseHostPort(hostport) + while(!driverUp) { + try { + val socket = new Socket(driverHost, driverPort) + socket.close() + logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) + driverUp = true + } catch { + case e: Exception => + logError("Failed to connect to driver at %s:%s, retrying ...". + format(driverHost, driverPort)) + Thread.sleep(100) + } + } + System.setProperty("spark.driver.host", driverHost) + System.setProperty("spark.driver.port", driverPort.toString) + + val driverUrl = "akka://spark@%s:%s/user/%s".format( + driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + + actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") + } + + + private def allocateWorkers() { + + // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. + val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = + scala.collection.immutable.Map() + + yarnAllocator = YarnAllocationHandler.newAllocator( + yarnConf, + amClient, + appAttemptId, + args, + preferredNodeLocationData) + + logInfo("Allocating " + args.numWorkers + " workers.") + // Wait until all containers have finished + // TODO: This is a bit ugly. Can we make it nicer? + // TODO: Handle container failure + + yarnAllocator.addResourceRequests(args.numWorkers) + while(yarnAllocator.getNumWorkersRunning < args.numWorkers) { + yarnAllocator.allocateResources() + Thread.sleep(100) + } + + logInfo("All workers have launched.") + + } + + // TODO: We might want to extend this to allocate more containers in case they die ! + private def launchReporterThread(_sleepTime: Long): Thread = { + val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime + + val t = new Thread { + override def run() { + while (!driverClosed) { + val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - + yarnAllocator.getNumPendingAllocate + if (missingWorkerCount > 0) { + logInfo("Allocating %d containers to make up for (potentially) lost containers". + format(missingWorkerCount)) + yarnAllocator.addResourceRequests(missingWorkerCount) + } + sendProgress() + Thread.sleep(sleepTime) + } + } + } + // setting to daemon status, though this is usually not a good idea. + t.setDaemon(true) + t.start() + logInfo("Started progress reporter thread - sleep time : " + sleepTime) + t + } + + private def sendProgress() { + logDebug("Sending progress") + // simulated with an allocate request with no nodes requested ... + yarnAllocator.allocateResources() + } + + def finishApplicationMaster(status: FinalApplicationStatus) { + logInfo("finish ApplicationMaster with " + status) + amClient.unregisterApplicationMaster(status, "" /* appMessage */, "" /* appTrackingUrl */) + } + +} + + +object WorkerLauncher { + def main(argStrings: Array[String]) { + val args = new ApplicationMasterArguments(argStrings) + new WorkerLauncher(args).run() + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala new file mode 100644 index 0000000000000..9f5523c4b97a8 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import java.net.URI +import java.nio.ByteBuffer +import java.security.PrivilegedExceptionAction + +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils +import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.client.api.NMClient +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.ipc.YarnRPC +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} + +import org.apache.spark.Logging + + +class WorkerRunnable( + container: Container, + conf: Configuration, + masterAddress: String, + slaveId: String, + hostname: String, + workerMemory: Int, + workerCores: Int) + extends Runnable with Logging { + + var rpc: YarnRPC = YarnRPC.create(conf) + var nmClient: NMClient = _ + val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + + def run = { + logInfo("Starting Worker Container") + nmClient = NMClient.createNMClient() + nmClient.init(yarnConf) + nmClient.start() + startContainer + } + + def startContainer = { + logInfo("Setting up ContainerLaunchContext") + + val ctx = Records.newRecord(classOf[ContainerLaunchContext]) + .asInstanceOf[ContainerLaunchContext] + + val localResources = prepareLocalResources + ctx.setLocalResources(localResources) + + val env = prepareEnvironment + ctx.setEnvironment(env) + + // Extra options for the JVM + var JAVA_OPTS = "" + // Set the JVM memory + val workerMemoryString = workerMemory + "m" + JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " " + if (env.isDefinedAt("SPARK_JAVA_OPTS")) { + JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " + } + + JAVA_OPTS += " -Djava.io.tmpdir=" + + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " + + // Commenting it out for now - so that people can refer to the properties if required. Remove + // it once cpuset version is pushed out. + // The context is, default gc for server class machines end up using all cores to do gc - hence + // if there are multiple containers in same node, spark gc effects all other containers + // performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in + // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset + // of cores on a node. +/* + else { + // If no java_opts specified, default to using -XX:+CMSIncrementalMode + // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont + // want to mess with it. + // In our expts, using (default) throughput collector has severe perf ramnifications in + // multi-tennent machines + // The options are based on + // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline + JAVA_OPTS += " -XX:+UseConcMarkSweepGC " + JAVA_OPTS += " -XX:+CMSIncrementalMode " + JAVA_OPTS += " -XX:+CMSIncrementalPacing " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " + JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + } +*/ + + val credentials = UserGroupInformation.getCurrentUser().getCredentials() + val dob = new DataOutputBuffer() + credentials.writeTokenStorageToStream(dob) + ctx.setTokens(ByteBuffer.wrap(dob.getData())) + + var javaCommand = "java" + val javaHome = System.getenv("JAVA_HOME") + if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { + javaCommand = Environment.JAVA_HOME.$() + "/bin/java" + } + + val commands = List[String](javaCommand + + " -server " + + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. + // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in + // an inconsistent state. + // TODO: If the OOM is not recoverable by rescheduling it on different node, then do + // 'something' to fail job ... akin to blacklisting trackers in mapred ? + " -XX:OnOutOfMemoryError='kill %p' " + + JAVA_OPTS + + " org.apache.spark.executor.CoarseGrainedExecutorBackend " + + masterAddress + " " + + slaveId + " " + + hostname + " " + + workerCores + + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + logInfo("Setting up worker with commands: " + commands) + ctx.setCommands(commands) + + // Send the start request to the ContainerManager + nmClient.startContainer(container, ctx) + } + + private def setupDistributedCache( + file: String, + rtype: LocalResourceType, + localResources: HashMap[String, LocalResource], + timestamp: String, + size: String, + vis: String) = { + val uri = new URI(file) + val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + amJarRsrc.setType(rtype) + amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) + amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) + amJarRsrc.setTimestamp(timestamp.toLong) + amJarRsrc.setSize(size.toLong) + localResources(uri.getFragment()) = amJarRsrc + } + + def prepareLocalResources: HashMap[String, LocalResource] = { + logInfo("Preparing Local resources") + val localResources = HashMap[String, LocalResource]() + + if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { + val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') + val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') + val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',') + val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',') + for( i <- 0 to distFiles.length - 1) { + setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i), + fileSizes(i), visibilities(i)) + } + } + + if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) { + val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',') + val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',') + val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',') + val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',') + for( i <- 0 to distArchives.length - 1) { + setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources, + timeStamps(i), fileSizes(i), visibilities(i)) + } + } + + logInfo("Prepared Local resources " + localResources) + localResources + } + + def prepareEnvironment: HashMap[String, String] = { + val env = new HashMap[String, String]() + + Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env) + + // Allow users to specify some environment variables + Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) + + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + env + } + +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala new file mode 100644 index 0000000000000..dba0f7640e67c --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -0,0 +1,687 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import java.lang.{Boolean => JBoolean} +import java.util.{Collections, Set => JSet} +import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} + +import org.apache.spark.Logging +import org.apache.spark.scheduler.SplitInfo +import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend} +import org.apache.spark.util.Utils + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.ApplicationMasterProtocol +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId +import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus} +import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest} +import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.hadoop.yarn.util.{RackResolver, Records} + + +object AllocationType extends Enumeration ("HOST", "RACK", "ANY") { + type AllocationType = Value + val HOST, RACK, ANY = Value +} + +// TODO: +// Too many params. +// Needs to be mt-safe +// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should +// make it more proactive and decoupled. + +// Note that right now, we assume all node asks as uniform in terms of capabilities and priority +// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for +// more info on how we are requesting for containers. +private[yarn] class YarnAllocationHandler( + val conf: Configuration, + val amClient: AMRMClient[ContainerRequest], + val appAttemptId: ApplicationAttemptId, + val maxWorkers: Int, + val workerMemory: Int, + val workerCores: Int, + val preferredHostToCount: Map[String, Int], + val preferredRackToCount: Map[String, Int]) + extends Logging { + // These three are locked on allocatedHostToContainersMap. Complementary data structures + // allocatedHostToContainersMap : containers which are running : host, Set + // allocatedContainerToHostMap: container to host mapping. + private val allocatedHostToContainersMap = + new HashMap[String, collection.mutable.Set[ContainerId]]() + + private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() + + // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an + // allocated node) + // As with the two data structures above, tightly coupled with them, and to be locked on + // allocatedHostToContainersMap + private val allocatedRackCount = new HashMap[String, Int]() + + // Containers which have been released. + private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]() + // Containers to be released in next request to RM + private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] + + // Number of container requests that have been sent to, but not yet allocated by the + // ApplicationMaster. + private val numPendingAllocate = new AtomicInteger() + private val numWorkersRunning = new AtomicInteger() + // Used to generate a unique id per worker + private val workerIdCounter = new AtomicInteger() + private val lastResponseId = new AtomicInteger() + private val numWorkersFailed = new AtomicInteger() + + def getNumPendingAllocate: Int = numPendingAllocate.intValue + + def getNumWorkersRunning: Int = numWorkersRunning.intValue + + def getNumWorkersFailed: Int = numWorkersFailed.intValue + + def isResourceConstraintSatisfied(container: Container): Boolean = { + container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + } + + def releaseContainer(container: Container) { + val containerId = container.getId + pendingReleaseContainers.put(containerId, true) + amClient.releaseAssignedContainer(containerId) + } + + def allocateResources() { + // We have already set the container request. Poll the ResourceManager for a response. + // This doubles as a heartbeat if there are no pending container requests. + val progressIndicator = 0.1f + val allocateResponse = amClient.allocate(progressIndicator) + + val allocatedContainers = allocateResponse.getAllocatedContainers() + if (allocatedContainers.size > 0) { + var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size) + + if (numPendingAllocateNow < 0) { + numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow) + } + + logDebug(""" + Allocated containers: %d + Current worker count: %d + Containers released: %s + Containers to-be-released: %s + Cluster resources: %s + """.format( + allocatedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers, + allocateResponse.getAvailableResources)) + + val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() + + for (container <- allocatedContainers) { + if (isResourceConstraintSatisfied(container)) { + // Add the accepted `container` to the host's list of already accepted, + // allocated containers + val host = container.getNodeId.getHost + val containersForHost = hostToContainers.getOrElseUpdate(host, + new ArrayBuffer[Container]()) + containersForHost += container + } else { + // Release container, since it doesn't satisfy resource constraints. + releaseContainer(container) + } + } + + // Find the appropriate containers to use. + // TODO: Cleanup this group-by... + val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() + + for (candidateHost <- hostToContainers.keySet) { + val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) + val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) + + val remainingContainersOpt = hostToContainers.get(candidateHost) + assert(remainingContainersOpt.isDefined) + var remainingContainers = remainingContainersOpt.get + + if (requiredHostCount >= remainingContainers.size) { + // Since we have <= required containers, add all remaining containers to + // `dataLocalContainers`. + dataLocalContainers.put(candidateHost, remainingContainers) + // There are no more free containers remaining. + remainingContainers = null + } else if (requiredHostCount > 0) { + // Container list has more containers than we need for data locality. + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. + val (dataLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredHostCount) + dataLocalContainers.put(candidateHost, dataLocal) + + // Invariant: remainingContainers == remaining + + // YARN has a nasty habit of allocating a ton of containers on a host - discourage this. + // Add each container in `remaining` to list of containers to release. If we have an + // insufficient number of containers, then the next allocation cycle will reallocate + // (but won't treat it as data local). + // TODO(harvey): Rephrase this comment some more. + for (container <- remaining) releaseContainer(container) + remainingContainers = null + } + + // For rack local containers + if (remainingContainers != null) { + val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + if (rack != null) { + val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) + val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - + rackLocalContainers.getOrElse(rack, List()).size + + if (requiredRackCount >= remainingContainers.size) { + // Add all remaining containers to to `dataLocalContainers`. + dataLocalContainers.put(rack, remainingContainers) + remainingContainers = null + } else if (requiredRackCount > 0) { + // Container list has more containers that we need for data locality. + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. + val (rackLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredRackCount) + val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, + new ArrayBuffer[Container]()) + + existingRackLocal ++= rackLocal + + remainingContainers = remaining + } + } + } + + if (remainingContainers != null) { + // Not all containers have been consumed - add them to the list of off-rack containers. + offRackContainers.put(candidateHost, remainingContainers) + } + } + + // Now that we have split the containers into various groups, go through them in order: + // first host-local, then rack-local, and finally off-rack. + // Note that the list we create below tries to ensure that not all containers end up within + // a host if there is a sufficiently large number of hosts/containers. + val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size) + allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(dataLocalContainers) + allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(rackLocalContainers) + allocatedContainersToProcess ++= ClusterScheduler.prioritizeContainers(offRackContainers) + + // Run each of the allocated containers. + for (container <- allocatedContainersToProcess) { + val numWorkersRunningNow = numWorkersRunning.incrementAndGet() + val workerHostname = container.getNodeId.getHost + val containerId = container.getId + + val workerMemoryOverhead = (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + assert(container.getResource.getMemory >= workerMemoryOverhead) + + if (numWorkersRunningNow > maxWorkers) { + logInfo("""Ignoring container %s at host %s, since we already have the required number of + containers for it.""".format(containerId, workerHostname)) + releaseContainer(container) + numWorkersRunning.decrementAndGet() + } else { + val workerId = workerIdCounter.incrementAndGet().toString + val driverUrl = "akka://spark@%s:%s/user/%s".format( + System.getProperty("spark.driver.host"), + System.getProperty("spark.driver.port"), + CoarseGrainedSchedulerBackend.ACTOR_NAME) + + logInfo("Launching container %s for on host %s".format(containerId, workerHostname)) + + // To be safe, remove the container from `pendingReleaseContainers`. + pendingReleaseContainers.remove(containerId) + + val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) + allocatedHostToContainersMap.synchronized { + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, + new HashSet[ContainerId]()) + + containerSet += containerId + allocatedContainerToHostMap.put(containerId, workerHostname) + + if (rack != null) { + allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) + } + } + logInfo("Launching WorkerRunnable. driverUrl: %s, workerHostname: %s".format(driverUrl, workerHostname)) + val workerRunnable = new WorkerRunnable( + container, + conf, + driverUrl, + workerId, + workerHostname, + workerMemory, + workerCores) + new Thread(workerRunnable).start() + } + } + logDebug(""" + Finished allocating %s containers (from %s originally). + Current number of workers running: %d, + releasedContainerList: %s, + pendingReleaseContainers: %s + """.format( + allocatedContainersToProcess, + allocatedContainers, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers)) + } + + val completedContainers = allocateResponse.getCompletedContainersStatuses() + if (completedContainers.size > 0) { + logDebug("Completed %d containers".format(completedContainers.size)) + + for (completedContainer <- completedContainers) { + val containerId = completedContainer.getContainerId + + if (pendingReleaseContainers.containsKey(containerId)) { + // YarnAllocationHandler already marked the container for release, so remove it from + // `pendingReleaseContainers`. + pendingReleaseContainers.remove(containerId) + } else { + // Decrement the number of workers running. The next iteration of the ApplicationMaster's + // reporting thread will take care of allocating. + numWorkersRunning.decrementAndGet() + logInfo("Completed container %s (state: %s, exit status: %s)".format( + containerId, + completedContainer.getState, + completedContainer.getExitStatus())) + // Hadoop 2.2.X added a ContainerExitStatus we should switch to use + // there are some exit status' we shouldn't necessarily count against us, but for + // now I think its ok as none of the containers are expected to exit + if (completedContainer.getExitStatus() != 0) { + logInfo("Container marked as failed: " + containerId) + numWorkersFailed.incrementAndGet() + } + } + + allocatedHostToContainersMap.synchronized { + if (allocatedContainerToHostMap.containsKey(containerId)) { + val hostOpt = allocatedContainerToHostMap.get(containerId) + assert(hostOpt.isDefined) + val host = hostOpt.get + + val containerSetOpt = allocatedHostToContainersMap.get(host) + assert(containerSetOpt.isDefined) + val containerSet = containerSetOpt.get + + containerSet.remove(containerId) + if (containerSet.isEmpty) { + allocatedHostToContainersMap.remove(host) + } else { + allocatedHostToContainersMap.update(host, containerSet) + } + + allocatedContainerToHostMap.remove(containerId) + + // TODO: Move this part outside the synchronized block? + val rack = YarnAllocationHandler.lookupRack(conf, host) + if (rack != null) { + val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 + if (rackCount > 0) { + allocatedRackCount.put(rack, rackCount) + } else { + allocatedRackCount.remove(rack) + } + } + } + } + } + logDebug(""" + Finished processing %d completed containers. + Current number of workers running: %d, + releasedContainerList: %s, + pendingReleaseContainers: %s + """.format( + completedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers)) + } + } + + def createRackResourceRequests( + hostContainers: ArrayBuffer[ContainerRequest] + ): ArrayBuffer[ContainerRequest] = { + // Generate modified racks and new set of hosts under it before issuing requests. + val rackToCounts = new HashMap[String, Int]() + + for (container <- hostContainers) { + val candidateHost = container.getNodes.last + assert(YarnAllocationHandler.ANY_HOST != candidateHost) + + val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) + if (rack != null) { + var count = rackToCounts.getOrElse(rack, 0) + count += 1 + rackToCounts.put(rack, count) + } + } + + val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size) + for ((rack, count) <- rackToCounts) { + requestedContainers ++= createResourceRequests( + AllocationType.RACK, + rack, + count, + YarnAllocationHandler.PRIORITY) + } + + requestedContainers + } + + def allocatedContainersOnHost(host: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedHostToContainersMap.getOrElse(host, Set()).size + } + retval + } + + def allocatedContainersOnRack(rack: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedRackCount.getOrElse(rack, 0) + } + retval + } + + def addResourceRequests(numWorkers: Int) { + val containerRequests: List[ContainerRequest] = + if (numWorkers <= 0 || preferredHostToCount.isEmpty) { + logDebug("numWorkers: " + numWorkers + ", host preferences: " + + preferredHostToCount.isEmpty) + createResourceRequests( + AllocationType.ANY, + resource = null, + numWorkers, + YarnAllocationHandler.PRIORITY).toList + } else { + // Request for all hosts in preferred nodes and for numWorkers - + // candidates.size, request by default allocation policy. + val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) + for ((candidateHost, candidateCount) <- preferredHostToCount) { + val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) + + if (requiredCount > 0) { + hostContainerRequests ++= createResourceRequests( + AllocationType.HOST, + candidateHost, + requiredCount, + YarnAllocationHandler.PRIORITY) + } + } + val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( + hostContainerRequests).toList + + val anyContainerRequests = createResourceRequests( + AllocationType.ANY, + resource = null, + numWorkers, + YarnAllocationHandler.PRIORITY) + + val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( + hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size) + + containerRequestBuffer ++= hostContainerRequests + containerRequestBuffer ++= rackContainerRequests + containerRequestBuffer ++= anyContainerRequests + containerRequestBuffer.toList + } + + for (request <- containerRequests) { + amClient.addContainerRequest(request) + } + + if (numWorkers > 0) { + numPendingAllocate.addAndGet(numWorkers) + logInfo("Will Allocate %d worker containers, each with %d memory".format( + numWorkers, + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))) + } else { + logDebug("Empty allocation request ...") + } + + for (request <- containerRequests) { + val nodes = request.getNodes + var hostStr = if (nodes == null || nodes.isEmpty) { + "Any" + } else { + nodes.last + } + logInfo("Container request (host: %s, priority: %s, capability: %s".format( + hostStr, + request.getPriority().getPriority, + request.getCapability)) + } + } + + private def createResourceRequests( + requestType: AllocationType.AllocationType, + resource: String, + numWorkers: Int, + priority: Int + ): ArrayBuffer[ContainerRequest] = { + + // If hostname is specified, then we need at least two requests - node local and rack local. + // There must be a third request, which is ANY. That will be specially handled. + requestType match { + case AllocationType.HOST => { + assert(YarnAllocationHandler.ANY_HOST != resource) + val hostname = resource + val nodeLocal = constructContainerRequests( + Array(hostname), + racks = null, + numWorkers, + priority) + + // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. + YarnAllocationHandler.populateRackInfo(conf, hostname) + nodeLocal + } + case AllocationType.RACK => { + val rack = resource + constructContainerRequests(hosts = null, Array(rack), numWorkers, priority) + } + case AllocationType.ANY => constructContainerRequests( + hosts = null, racks = null, numWorkers, priority) + case _ => throw new IllegalArgumentException( + "Unexpected/unsupported request type: " + requestType) + } + } + + private def constructContainerRequests( + hosts: Array[String], + racks: Array[String], + numWorkers: Int, + priority: Int + ): ArrayBuffer[ContainerRequest] = { + + val memoryResource = Records.newRecord(classOf[Resource]) + memoryResource.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + + val prioritySetting = Records.newRecord(classOf[Priority]) + prioritySetting.setPriority(priority) + + val requests = new ArrayBuffer[ContainerRequest]() + for (i <- 0 until numWorkers) { + requests += new ContainerRequest(memoryResource, hosts, racks, prioritySetting) + } + requests + } +} + +object YarnAllocationHandler { + + val ANY_HOST = "*" + // All requests are issued with same priority : we do not (yet) have any distinction between + // request types (like map/reduce in hadoop for example) + val PRIORITY = 1 + + // Additional memory overhead - in mb. + val MEMORY_OVERHEAD = 384 + + // Host to rack map - saved from allocation requests. We are expecting this not to change. + // Note that it is possible for this to change : and ResurceManager will indicate that to us via + // update response to allocate. But we are punting on handling that for now. + private val hostToRack = new ConcurrentHashMap[String, String]() + private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() + + + def newAllocator( + conf: Configuration, + amClient: AMRMClient[ContainerRequest], + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments + ): YarnAllocationHandler = { + new YarnAllocationHandler( + conf, + amClient, + appAttemptId, + args.numWorkers, + args.workerMemory, + args.workerCores, + Map[String, Int](), + Map[String, Int]()) + } + + def newAllocator( + conf: Configuration, + amClient: AMRMClient[ContainerRequest], + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments, + map: collection.Map[String, + collection.Set[SplitInfo]] + ): YarnAllocationHandler = { + val (hostToSplitCount, rackToSplitCount) = generateNodeToWeight(conf, map) + new YarnAllocationHandler( + conf, + amClient, + appAttemptId, + args.numWorkers, + args.workerMemory, + args.workerCores, + hostToSplitCount, + rackToSplitCount) + } + + def newAllocator( + conf: Configuration, + amClient: AMRMClient[ContainerRequest], + appAttemptId: ApplicationAttemptId, + maxWorkers: Int, + workerMemory: Int, + workerCores: Int, + map: collection.Map[String, collection.Set[SplitInfo]] + ): YarnAllocationHandler = { + val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) + new YarnAllocationHandler( + conf, + amClient, + appAttemptId, + maxWorkers, + workerMemory, + workerCores, + hostToCount, + rackToCount) + } + + // A simple method to copy the split info map. + private def generateNodeToWeight( + conf: Configuration, + input: collection.Map[String, collection.Set[SplitInfo]] + ): (Map[String, Int], Map[String, Int]) = { + + if (input == null) { + return (Map[String, Int](), Map[String, Int]()) + } + + val hostToCount = new HashMap[String, Int] + val rackToCount = new HashMap[String, Int] + + for ((host, splits) <- input) { + val hostCount = hostToCount.getOrElse(host, 0) + hostToCount.put(host, hostCount + splits.size) + + val rack = lookupRack(conf, host) + if (rack != null){ + val rackCount = rackToCount.getOrElse(host, 0) + rackToCount.put(host, rackCount + splits.size) + } + } + + (hostToCount.toMap, rackToCount.toMap) + } + + def lookupRack(conf: Configuration, host: String): String = { + if (!hostToRack.contains(host)) { + populateRackInfo(conf, host) + } + hostToRack.get(host) + } + + def fetchCachedHostsForRack(rack: String): Option[Set[String]] = { + Option(rackToHostSet.get(rack)).map { set => + val convertedSet: collection.mutable.Set[String] = set + // TODO: Better way to get a Set[String] from JSet. + convertedSet.toSet + } + } + + def populateRackInfo(conf: Configuration, hostname: String) { + Utils.checkHost(hostname) + + if (!hostToRack.containsKey(hostname)) { + // If there are repeated failures to resolve, all to an ignore list. + val rackInfo = RackResolver.resolve(conf, hostname) + if (rackInfo != null && rackInfo.getNetworkLocation != null) { + val rack = rackInfo.getNetworkLocation + hostToRack.put(hostname, rack) + if (! rackToHostSet.containsKey(rack)) { + rackToHostSet.putIfAbsent(rack, + Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]())) + } + rackToHostSet.get(rack).add(hostname) + + // TODO(harvey): Figure out what this comment means... + // Since RackResolver caches, we are disabling this for now ... + } /* else { + // right ? Else we will keep calling rack resolver in case we cant resolve rack info ... + hostToRack.put(hostname, null) + } */ + } + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala new file mode 100644 index 0000000000000..2ba2366ead171 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.conf.Configuration + +/** + * Contains util methods to interact with Hadoop from spark. + */ +class YarnSparkHadoopUtil extends SparkHadoopUtil { + + // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true. + override def isYarnMode(): Boolean = { true } + + // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems + // Always create a new config, dont reuse yarnConf. + override def newConfiguration(): Configuration = new YarnConfiguration(new Configuration()) + + // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + override def addCredentials(conf: JobConf) { + val jobCreds = conf.getCredentials() + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala new file mode 100644 index 0000000000000..63a0449e5a073 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.scheduler.cluster + +import org.apache.spark._ +import org.apache.hadoop.conf.Configuration +import org.apache.spark.deploy.yarn.YarnAllocationHandler +import org.apache.spark.util.Utils + +/** + * + * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM. + */ +private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) { + + def this(sc: SparkContext) = this(sc, new Configuration()) + + // By default, rack is unknown + override def getRackForHost(hostPort: String): Option[String] = { + val host = Utils.parseHostPort(hostPort)._1 + val retval = YarnAllocationHandler.lookupRack(conf, host) + if (retval != null) Some(retval) else None + } + + override def postStartHook() { + + // The yarn application is running, but the worker might not yet ready + // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt + Thread.sleep(2000L) + logInfo("YarnClientClusterScheduler.postStartHook done") + } +} diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala new file mode 100644 index 0000000000000..b206780c7806e --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.scheduler.cluster + +import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} +import org.apache.spark.{SparkException, Logging, SparkContext} +import org.apache.spark.deploy.yarn.{Client, ClientArguments} + +private[spark] class YarnClientSchedulerBackend( + scheduler: ClusterScheduler, + sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + with Logging { + + var client: Client = null + var appId: ApplicationId = null + + override def start() { + super.start() + + val defalutWorkerCores = "2" + val defalutWorkerMemory = "512m" + val defaultWorkerNumber = "1" + + val userJar = System.getenv("SPARK_YARN_APP_JAR") + var workerCores = System.getenv("SPARK_WORKER_CORES") + var workerMemory = System.getenv("SPARK_WORKER_MEMORY") + var workerNumber = System.getenv("SPARK_WORKER_INSTANCES") + + if (userJar == null) + throw new SparkException("env SPARK_YARN_APP_JAR is not set") + + if (workerCores == null) + workerCores = defalutWorkerCores + if (workerMemory == null) + workerMemory = defalutWorkerMemory + if (workerNumber == null) + workerNumber = defaultWorkerNumber + + val driverHost = System.getProperty("spark.driver.host") + val driverPort = System.getProperty("spark.driver.port") + val hostport = driverHost + ":" + driverPort + + val argsArray = Array[String]( + "--class", "notused", + "--jar", userJar, + "--args", hostport, + "--worker-memory", workerMemory, + "--worker-cores", workerCores, + "--num-workers", workerNumber, + "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" + ) + + val args = new ClientArguments(argsArray) + client = new Client(args) + appId = client.runApp() + waitForApp() + } + + def waitForApp() { + + // TODO : need a better way to find out whether the workers are ready or not + // maybe by resource usage report? + while(true) { + val report = client.getApplicationReport(appId) + + logInfo("Application report from ASM: \n" + + "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + + "\t appStartTime: " + report.getStartTime() + "\n" + + "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + ) + + // Ready to go, or already gone. + val state = report.getYarnApplicationState() + if (state == YarnApplicationState.RUNNING) { + return + } else if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + throw new SparkException("Yarn application already ended," + + "might be killed or not able to launch application master.") + } + + Thread.sleep(1000) + } + } + + override def stop() { + super.stop() + client.stop() + logInfo("Stoped") + } + +} diff --git a/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala new file mode 100644 index 0000000000000..29b3f22e13697 --- /dev/null +++ b/new-yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.scheduler.cluster + +import org.apache.spark._ +import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler} +import org.apache.spark.util.Utils +import org.apache.hadoop.conf.Configuration + +/** + * + * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of ApplicationMaster, etc is done + */ +private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) { + + logInfo("Created YarnClusterScheduler") + + def this(sc: SparkContext) = this(sc, new Configuration()) + + // Nothing else for now ... initialize application master : which needs sparkContext to determine how to allocate + // Note that only the first creation of SparkContext influences (and ideally, there must be only one SparkContext, right ?) + // Subsequent creations are ignored - since nodes are already allocated by then. + + + // By default, rack is unknown + override def getRackForHost(hostPort: String): Option[String] = { + val host = Utils.parseHostPort(hostPort)._1 + val retval = YarnAllocationHandler.lookupRack(conf, host) + if (retval != null) Some(retval) else None + } + + override def postStartHook() { + val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc) + if (sparkContextInitialized){ + // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt + Thread.sleep(3000L) + } + logInfo("YarnClusterScheduler.postStartHook done") + } +} diff --git a/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala new file mode 100644 index 0000000000000..2941356bc55f9 --- /dev/null +++ b/new-yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.yarn + +import java.net.URI + +import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar +import org.mockito.Mockito.when + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.permission.FsAction +import org.apache.hadoop.yarn.api.records.LocalResource +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility +import org.apache.hadoop.yarn.api.records.LocalResourceType +import org.apache.hadoop.yarn.util.{Records, ConverterUtils} + +import scala.collection.mutable.HashMap +import scala.collection.mutable.Map + + +class ClientDistributedCacheManagerSuite extends FunSuite with MockitoSugar { + + class MockClientDistributedCacheManager extends ClientDistributedCacheManager { + override def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): + LocalResourceVisibility = { + return LocalResourceVisibility.PRIVATE + } + } + + test("test getFileStatus empty") { + val distMgr = new ClientDistributedCacheManager() + val fs = mock[FileSystem] + val uri = new URI("/tmp/testing") + when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus()) + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + val stat = distMgr.getFileStatus(fs, uri, statCache) + assert(stat.getPath() === null) + } + + test("test getFileStatus cached") { + val distMgr = new ClientDistributedCacheManager() + val fs = mock[FileSystem] + val uri = new URI("/tmp/testing") + val realFileStatus = new FileStatus(10, false, 1, 1024, 10, 10, null, "testOwner", + null, new Path("/tmp/testing")) + when(fs.getFileStatus(new Path(uri))).thenReturn(new FileStatus()) + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus](uri -> realFileStatus) + val stat = distMgr.getFileStatus(fs, uri, statCache) + assert(stat.getPath().toString() === "/tmp/testing") + } + + test("test addResource") { + val distMgr = new MockClientDistributedCacheManager() + val fs = mock[FileSystem] + val conf = new Configuration() + val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") + val localResources = HashMap[String, LocalResource]() + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + when(fs.getFileStatus(destPath)).thenReturn(new FileStatus()) + + distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, "link", + statCache, false) + val resource = localResources("link") + assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) + assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) + assert(resource.getTimestamp() === 0) + assert(resource.getSize() === 0) + assert(resource.getType() === LocalResourceType.FILE) + + val env = new HashMap[String, String]() + distMgr.setDistFilesEnv(env) + assert(env("SPARK_YARN_CACHE_FILES") === "file:/foo.invalid.com:8080/tmp/testing#link") + assert(env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === "0") + assert(env("SPARK_YARN_CACHE_FILES_FILE_SIZES") === "0") + assert(env("SPARK_YARN_CACHE_FILES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) + + distMgr.setDistArchivesEnv(env) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) + + //add another one and verify both there and order correct + val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", + null, new Path("/tmp/testing2")) + val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2") + when(fs.getFileStatus(destPath2)).thenReturn(realFileStatus) + distMgr.addResource(fs, conf, destPath2, localResources, LocalResourceType.FILE, "link2", + statCache, false) + val resource2 = localResources("link2") + assert(resource2.getVisibility() === LocalResourceVisibility.PRIVATE) + assert(ConverterUtils.getPathFromYarnURL(resource2.getResource()) === destPath2) + assert(resource2.getTimestamp() === 10) + assert(resource2.getSize() === 20) + assert(resource2.getType() === LocalResourceType.FILE) + + val env2 = new HashMap[String, String]() + distMgr.setDistFilesEnv(env2) + val timestamps = env2("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') + val files = env2("SPARK_YARN_CACHE_FILES").split(',') + val sizes = env2("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') + val visibilities = env2("SPARK_YARN_CACHE_FILES_VISIBILITIES") .split(',') + assert(files(0) === "file:/foo.invalid.com:8080/tmp/testing#link") + assert(timestamps(0) === "0") + assert(sizes(0) === "0") + assert(visibilities(0) === LocalResourceVisibility.PRIVATE.name()) + + assert(files(1) === "file:/foo.invalid.com:8080/tmp/testing2#link2") + assert(timestamps(1) === "10") + assert(sizes(1) === "20") + assert(visibilities(1) === LocalResourceVisibility.PRIVATE.name()) + } + + test("test addResource link null") { + val distMgr = new MockClientDistributedCacheManager() + val fs = mock[FileSystem] + val conf = new Configuration() + val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") + val localResources = HashMap[String, LocalResource]() + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + when(fs.getFileStatus(destPath)).thenReturn(new FileStatus()) + + intercept[Exception] { + distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, null, + statCache, false) + } + assert(localResources.get("link") === None) + assert(localResources.size === 0) + } + + test("test addResource appmaster only") { + val distMgr = new MockClientDistributedCacheManager() + val fs = mock[FileSystem] + val conf = new Configuration() + val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") + val localResources = HashMap[String, LocalResource]() + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", + null, new Path("/tmp/testing")) + when(fs.getFileStatus(destPath)).thenReturn(realFileStatus) + + distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", + statCache, true) + val resource = localResources("link") + assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) + assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) + assert(resource.getTimestamp() === 10) + assert(resource.getSize() === 20) + assert(resource.getType() === LocalResourceType.ARCHIVE) + + val env = new HashMap[String, String]() + distMgr.setDistFilesEnv(env) + assert(env.get("SPARK_YARN_CACHE_FILES") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) + + distMgr.setDistArchivesEnv(env) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) + assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) + } + + test("test addResource archive") { + val distMgr = new MockClientDistributedCacheManager() + val fs = mock[FileSystem] + val conf = new Configuration() + val destPath = new Path("file:///foo.invalid.com:8080/tmp/testing") + val localResources = HashMap[String, LocalResource]() + val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() + val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", + null, new Path("/tmp/testing")) + when(fs.getFileStatus(destPath)).thenReturn(realFileStatus) + + distMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE, "link", + statCache, false) + val resource = localResources("link") + assert(resource.getVisibility() === LocalResourceVisibility.PRIVATE) + assert(ConverterUtils.getPathFromYarnURL(resource.getResource()) === destPath) + assert(resource.getTimestamp() === 10) + assert(resource.getSize() === 20) + assert(resource.getType() === LocalResourceType.ARCHIVE) + + val env = new HashMap[String, String]() + + distMgr.setDistArchivesEnv(env) + assert(env("SPARK_YARN_CACHE_ARCHIVES") === "file:/foo.invalid.com:8080/tmp/testing#link") + assert(env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") === "10") + assert(env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === "20") + assert(env("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === LocalResourceVisibility.PRIVATE.name()) + + distMgr.setDistFilesEnv(env) + assert(env.get("SPARK_YARN_CACHE_FILES") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_TIME_STAMPS") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_FILE_SIZES") === None) + assert(env.get("SPARK_YARN_CACHE_FILES_VISIBILITIES") === None) + } + + +} diff --git a/pom.xml b/pom.xml index 07213f9d3015f..83f9b23e12de2 100644 --- a/pom.xml +++ b/pom.xml @@ -102,7 +102,9 @@ 1.5 2.9.3 0.13.0 + com.typesafe.akka 2.0.5 + 2.4.1 1.7.2 1.2.17 1.0.4 @@ -234,7 +236,7 @@ com.google.protobuf protobuf-java - 2.4.1 + ${protobuf.version} com.twitter @@ -247,20 +249,31 @@ 0.3.1 - com.typesafe.akka + ${akka.group} akka-actor ${akka.version} - com.typesafe.akka + ${akka.group} akka-remote ${akka.version} - com.typesafe.akka + ${akka.group} akka-slf4j ${akka.version} + + ${akka.group} + akka-zeromq + ${akka.version} + + + org.jboss.netty + netty + + + it.unimi.dsi fastutil @@ -271,11 +284,6 @@ colt 1.2.0 - - com.github.scala-incubator.io - scala-io-file_2.9.2 - 0.4.1 - org.apache.mesos mesos @@ -768,6 +776,41 @@ + + + new-yarn + + org.spark-project + 2.0.5-protobuf-2.5-java-1.5 + 2 + 2.2.0 + 2.5.0 + + + + new-yarn + + + + + maven-root + Maven root repository + http://repo1.maven.org/maven2/ + + true + + + false + + + + + + + + + + repl-bin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 660f0e2eff822..03d4cae759bbf 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -28,6 +28,11 @@ object SparkBuild extends Build { // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set // through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN. val DEFAULT_HADOOP_VERSION = "1.0.4" + + // Whether the Hadoop version to build against is 2.2.x, or a variant of it. This can be set + // through the SPARK_IS_NEW_HADOOP environment variable. + val DEFAULT_IS_NEW_HADOOP = false + val DEFAULT_YARN = false // HBase version; set as appropriate. @@ -55,8 +60,6 @@ object SparkBuild extends Build { lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) - lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core) - lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) @@ -66,14 +69,28 @@ object SparkBuild extends Build { // Allows build configuration to be set through environment variables lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) + lazy val isNewHadoop = scala.util.Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { + case None => { + val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined + (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP) + } + case Some(v) => v.toBoolean + } lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match { case None => DEFAULT_YARN case Some(v) => v.toBoolean } + // Build against a protobuf-2.5 compatible Akka if Hadoop 2 is used. + lazy val protobufVersion = if (isNewHadoop) "2.5.0" else "2.4.1" + lazy val akkaVersion = if (isNewHadoop) "2.0.5-protobuf-2.5-java-1.5" else "2.0.5" + lazy val akkaGroup = if (isNewHadoop) "org.spark-project" else "com.typesafe.akka" + // Conditionally include the yarn sub-project - lazy val maybeYarn = if(isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() - lazy val maybeYarnRef = if(isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]() + lazy val yarn = Project("yarn", file(if (isNewHadoop) "new-yarn" else "yarn"), settings = yarnSettings) dependsOn(core) + lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() + lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]() + lazy val allProjects = Seq[ProjectReference]( core, repl, examples, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef @@ -203,10 +220,10 @@ object SparkBuild extends Build { "com.ning" % "compress-lzf" % "0.8.4", "org.xerial.snappy" % "snappy-java" % "1.0.5", "org.ow2.asm" % "asm" % "4.0", - "com.google.protobuf" % "protobuf-java" % "2.4.1", - "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty), + "com.google.protobuf" % "protobuf-java" % protobufVersion, + akkaGroup % "akka-actor" % akkaVersion excludeAll(excludeNetty), + akkaGroup % "akka-remote" % akkaVersion excludeAll(excludeNetty), + akkaGroup % "akka-slf4j" % akkaVersion excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", @@ -280,7 +297,7 @@ object SparkBuild extends Build { "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty) + akkaGroup % "akka-zeromq" % akkaVersion excludeAll(excludeNetty) ) ) diff --git a/streaming/pom.xml b/streaming/pom.xml index cf5de8f9a4bd9..320d5326f3819 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -86,9 +86,8 @@ scala-library - com.typesafe.akka + ${akka.group} akka-zeromq - 2.0.3 org.scalatest diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a7baf0c36cfd4..240ed8b32ae93 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -22,9 +22,12 @@ import java.net.Socket import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} +import scala.collection.JavaConversions._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ @@ -32,38 +35,38 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} + import org.apache.spark.{SparkContext, Logging} import org.apache.spark.util.Utils -import scala.collection.JavaConversions._ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging { def this(args: ApplicationMasterArguments) = this(args, new Configuration()) private var rpc: YarnRPC = YarnRPC.create(conf) - private var resourceManager: AMRMProtocol = null - private var appAttemptId: ApplicationAttemptId = null - private var userThread: Thread = null + private var resourceManager: AMRMProtocol = _ + private var appAttemptId: ApplicationAttemptId = _ + private var userThread: Thread = _ private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private val fs = FileSystem.get(yarnConf) - private var yarnAllocator: YarnAllocationHandler = null - private var isFinished:Boolean = false - private var uiAddress: String = "" + private var yarnAllocator: YarnAllocationHandler = _ + private var isFinished: Boolean = false + private var uiAddress: String = _ private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true - // default to numWorkers * 2, with minimum of 3 + // default to numWorkers * 2, with minimum of 3 private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3).toString()).toInt def run() { - // setup the directories so things go to yarn approved directories rather - // then user specified and /tmp + // Setup the directories so things go to yarn approved directories rather + // then user specified and /tmp. System.setProperty("spark.local.dir", getLocalDirs()) - // use priority 30 as its higher then HDFS. Its same priority as MapReduce is using + // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) appAttemptId = getApplicationAttemptId() @@ -72,9 +75,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Workaround until hadoop moves to something which has // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line) - // ignore result + // ignore result. // This does not, unfortunately, always work reliably ... but alleviates the bug a lot of times - // Hence args.workerCores = numCore disabled above. Any better option ? + // Hence args.workerCores = numCore disabled above. Any better option? // Compute number of threads for akka //val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() @@ -100,7 +103,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e waitForSparkContextInitialized() - // do this after spark master is up and SparkContext is created so that we can register UI Url + // Do this after spark master is up and SparkContext is created so that we can register UI Url val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() // Allocate all containers @@ -119,12 +122,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) + .getOrElse("")) if (localDirs.isEmpty()) { throw new Exception("Yarn Local dirs can't be empty") } - return localDirs + localDirs } private def getApplicationAttemptId(): ApplicationAttemptId = { @@ -133,7 +136,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val containerId = ConverterUtils.toContainerId(containerIdString) val appAttemptId = containerId.getApplicationAttemptId() logInfo("ApplicationAttemptId: " + appAttemptId) - return appAttemptId + appAttemptId } private def registerWithResourceManager(): AMRMProtocol = { @@ -141,7 +144,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e YarnConfiguration.RM_SCHEDULER_ADDRESS, YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) logInfo("Connecting to ResourceManager at " + rmAddress) - return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] + rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] } private def registerApplicationMaster(): RegisterApplicationMasterResponse = { @@ -149,12 +152,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) .asInstanceOf[RegisterApplicationMasterRequest] appMasterRequest.setApplicationAttemptId(appAttemptId) - // Setting this to master host,port - so that the ApplicationReport at client has some sensible info. + // Setting this to master host,port - so that the ApplicationReport at client has some + // sensible info. // Users can then monitor stderr/stdout on that node if required. appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) appMasterRequest.setTrackingUrl(uiAddress) - return resourceManager.registerApplicationMaster(appMasterRequest) + resourceManager.registerApplicationMaster(appMasterRequest) } private def waitForSparkMaster() { @@ -168,21 +172,25 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e try { val socket = new Socket(driverHost, driverPort.toInt) socket.close() - logInfo("Driver now available: " + driverHost + ":" + driverPort) + logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) driverUp = true } catch { - case e: Exception => - logWarning("Failed to connect to driver at " + driverHost + ":" + driverPort + ", retrying") - Thread.sleep(100) - tries = tries + 1 + case e: Exception => { + logWarning("Failed to connect to driver at %s:%s, retrying ...". + format(driverHost, driverPort)) + Thread.sleep(100) + tries = tries + 1 + } } } } private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") - val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader) - .getMethod("main", classOf[Array[String]]) + val mainMethod = Class.forName( + args.userClass, + false /* initialize */, + Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { var successed = false @@ -207,7 +215,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } t.start() - return t + t } // this need to happen before allocateWorkers @@ -229,13 +237,20 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e if (null != sparkContext) { uiAddress = sparkContext.ui.appUIAddress - this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, - appAttemptId, args, sparkContext.preferredNodeLocationData) + this.yarnAllocator = YarnAllocationHandler.newAllocator( + yarnConf, + resourceManager, + appAttemptId, + args, + sparkContext.preferredNodeLocationData) } else { - logWarning("Unable to retrieve sparkContext inspite of waiting for " + count * waitTime + - ", numTries = " + numTries) - this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, - appAttemptId, args) + logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d". + format(count * waitTime, numTries)) + this.yarnAllocator = YarnAllocationHandler.newAllocator( + yarnConf, + resourceManager, + appAttemptId, + args) } } } finally { @@ -251,36 +266,39 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - while(yarnAllocator.getNumWorkersRunning < args.numWorkers && - // If user thread exists, then quit ! - userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") - } - yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) - ApplicationMaster.incrementAllocatorLoop(1) - Thread.sleep(100) + + // Exists the loop if the user thread exits. + while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) { + if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of worker failures reached") + } + yarnAllocator.allocateContainers( + math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + ApplicationMaster.incrementAllocatorLoop(1) + Thread.sleep(100) } } finally { - // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT : - // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks + // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT, + // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } logInfo("All workers have launched.") - // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout + // Launch a progress reporter thread, else the app will get killed after expiration + // (def: 10mins) timeout. + // TODO(harvey): Verify the timeout if (userThread.isAlive) { - // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. - + // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses. val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) // we want to be reasonably responsive without causing too many requests to RM. - val schedulerInterval = + val schedulerInterval = System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) + launchReporterThread(interval) } } @@ -292,12 +310,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e override def run() { while (userThread.isAlive) { if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, + finishApplicationMaster(FinalApplicationStatus.FAILED, "max number of worker failures reached") } val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning if (missingWorkerCount > 0) { - logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") + logInfo("Allocating %d containers to make up for (potentially) lost containers". + format(missingWorkerCount)) yarnAllocator.allocateContainers(missingWorkerCount) } else sendProgress() @@ -305,16 +324,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } } - // setting to daemon status, though this is usually not a good idea. + // Setting to daemon status, though this is usually not a good idea. t.setDaemon(true) t.start() logInfo("Started progress reporter thread - sleep time : " + sleepTime) - return t + t } private def sendProgress() { logDebug("Sending progress") - // simulated with an allocate request with no nodes requested ... + // Simulated with an allocate request with no nodes requested ... yarnAllocator.allocateContainers(0) } @@ -334,7 +353,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e */ def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") { - synchronized { if (isFinished) { return @@ -348,14 +366,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e finishReq.setAppAttemptId(appAttemptId) finishReq.setFinishApplicationStatus(status) finishReq.setDiagnostics(diagnostics) - // set tracking url to empty since we don't have a history server + // Set tracking url to empty since we don't have a history server. finishReq.setTrackingUrl("") resourceManager.finishApplicationMaster(finishReq) - } /** - * clean up the staging directory. + * Clean up the staging directory. */ private def cleanupStagingDir() { var stagingDirPath: Path = null @@ -371,13 +388,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e fs.delete(stagingDirPath, true) } } catch { - case e: IOException => - logError("Failed to cleanup staging dir " + stagingDirPath, e) + case ioe: IOException => + logError("Failed to cleanup staging dir " + stagingDirPath, ioe) } } - // The shutdown hook that runs when a signal is received AND during normal - // close of the JVM. + // The shutdown hook that runs when a signal is received AND during normal close of the JVM. class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable { def run() { @@ -387,15 +403,14 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir() } } - } object ApplicationMaster { - // number of times to wait for the allocator loop to complete. - // each loop iteration waits for 100ms, so maximum of 3 seconds. + // Number of times to wait for the allocator loop to complete. + // Each loop iteration waits for 100ms, so maximum of 3 seconds. // This is to ensure that we have reasonable number of containers before we start - // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be optimal as more - // containers are available. Might need to handle this better. + // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be + // optimal as more containers are available. Might need to handle this better. private val ALLOCATOR_LOOP_WAIT_COUNT = 30 def incrementAllocatorLoop(by: Int) { val count = yarnAllocatorLoop.getAndAdd(by) @@ -413,7 +428,8 @@ object ApplicationMaster { applicationMasters.add(master) } - val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) + val sparkContextRef: AtomicReference[SparkContext] = + new AtomicReference[SparkContext](null /* initialValue */) val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0) def sparkContextInitialized(sc: SparkContext): Boolean = { @@ -423,19 +439,21 @@ object ApplicationMaster { sparkContextRef.notifyAll() } - // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do System.exit - // Should not really have to do this, but it helps yarn to evict resources earlier. - // not to mention, prevent Client declaring failure even though we exit'ed properly. - // Note that this will unfortunately not properly clean up the staging files because it gets called to - // late and the filesystem is already shutdown. + // Add a shutdown hook - as a best case effort in case users do not call sc.stop or do + // System.exit. + // Should not really have to do this, but it helps YARN to evict resources earlier. + // Not to mention, prevent the Client from declaring failure even though we exited properly. + // Note that this will unfortunately not properly clean up the staging files because it gets + // called too late, after the filesystem is already shutdown. if (modified) { Runtime.getRuntime().addShutdownHook(new Thread with Logging { - // This is not just to log, but also to ensure that log system is initialized for this instance when we actually are 'run' + // This is not only logs, but also ensures that log system is initialized for this instance + // when we are actually 'run'-ing. logInfo("Adding shutdown hook for context " + sc) override def run() { logInfo("Invoking sc stop from shutdown hook") sc.stop() - // best case ... + // Best case ... for (master <- applicationMasters) { master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } @@ -443,7 +461,7 @@ object ApplicationMaster { } ) } - // Wait for initialization to complete and atleast 'some' nodes can get allocated + // Wait for initialization to complete and atleast 'some' nodes can get allocated. yarnAllocatorLoop.synchronized { while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) { yarnAllocatorLoop.wait(1000L) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index bb73f6d337ba0..79dd03806523b 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -20,41 +20,46 @@ package org.apache.spark.deploy.yarn import java.net.{InetAddress, UnknownHostException, URI} import java.nio.ByteBuffer +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap +import scala.collection.mutable.Map + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil} -import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.mapred.Master +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.mapred.Master +import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ +import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.YarnClientImpl import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map -import scala.collection.JavaConversions._ +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.Logging class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { - + def this(args: ClientArguments) = this(new Configuration(), args) - + var rpc: YarnRPC = YarnRPC.create(conf) val yarnConf: YarnConfiguration = new YarnConfiguration(conf) val credentials = UserGroupInformation.getCurrentUser().getCredentials() private val SPARK_STAGING: String = ".sparkStaging" private val distCacheMgr = new ClientDistributedCacheManager() - // staging directory is private! -> rwx-------- + // Staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short) - // app files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) + + // App files are world-wide readable and owner writable -> rw-r--r-- + val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short) // for client user who want to monitor app status by itself. def runApp() = { @@ -89,15 +94,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } def validateArgs() = { - Map((System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", + Map( + (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", (args.userJar == null) -> "Error: You must specify a user jar!", (args.userClass == null) -> "Error: You must specify a user class!", (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!", - (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> - ("Error: AM memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD), - (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> - ("Error: Worker memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString())) - .foreach { case(cond, errStr) => + (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be " + + "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), + (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " + + "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD) + ).foreach { case(cond, errStr) => if (cond) { logError(errStr) args.printUsageAndExit(1) @@ -111,19 +117,24 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl def logClusterResourceDetails() { val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics - logInfo("Got Cluster metric info from ASM, numNodeManagers=" + clusterMetrics.getNumNodeManagers) + logInfo("Got Cluster metric info from ASM, numNodeManagers = " + + clusterMetrics.getNumNodeManagers) val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue) - logInfo("Queue info .. queueName=" + queueInfo.getQueueName + ", queueCurrentCapacity=" + queueInfo.getCurrentCapacity + - ", queueMaxCapacity=" + queueInfo.getMaximumCapacity + ", queueApplicationCount=" + queueInfo.getApplications.size + - ", queueChildQueueCount=" + queueInfo.getChildQueues.size) + logInfo("""Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s, + queueApplicationCount = %s, queueChildQueueCount = %s""".format( + queueInfo.getQueueName, + queueInfo.getCurrentCapacity, + queueInfo.getMaximumCapacity, + queueInfo.getApplications.size, + queueInfo.getChildQueues.size)) } - + def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) - - // if we have requested more then the clusters max for a single resource then exit. + + // If we have requested more then the clusters max for a single resource then exit. if (args.workerMemory > maxMem) { logError("the worker size is to large to run on this cluster " + args.workerMemory) System.exit(1) @@ -134,10 +145,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl System.exit(1) } - // We could add checks to make sure the entire cluster has enough resources but that involves getting - // all the node reports and computing ourselves + // We could add checks to make sure the entire cluster has enough resources but that involves + // getting all the node reports and computing ourselves } - + def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { logInfo("Setting up application submission context for ASM") val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) @@ -146,9 +157,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl return appContext } - /* - * see if two file systems are the same or not. - */ + /** See if two file systems are the same or not. */ private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { val srcUri = srcFs.getUri() val dstUri = destFs.getUri() @@ -183,9 +192,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl return true } - /** - * Copy the file into HDFS if needed. - */ + /** Copy the file into HDFS if needed. */ private def copyRemoteFile( dstDir: Path, originalPath: Path, @@ -201,9 +208,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl fs.setReplication(newPath, replication) if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) } - // resolve any symlinks in the URI path so using a "current" symlink - // to point to a specific version shows the specific version - // in the distributed cache configuration + // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific + // version shows the specific version in the distributed cache configuration val qualPath = fs.makeQualified(newPath) val fc = FileContext.getFileContext(qualPath.toUri(), conf) val destPath = fc.resolvePath(qualPath) @@ -212,8 +218,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { logInfo("Preparing Local resources") - // Upload Spark and the application JAR to the remote file system if necessary - // Add them as local resources to the AM + // Upload Spark and the application JAR to the remote file system if necessary. Add them as + // local resources to the AM. val fs = FileSystem.get(conf) val delegTokenRenewer = Master.getMasterPrincipal(conf) @@ -243,7 +249,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl var localURI = new URI(localPath) // if not specified assume these are in the local filesystem to keep behavior like Hadoop if (localURI.getScheme() == null) { - localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString()) + localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString) } val setPermissions = if (destName.equals(Client.APP_JAR)) true else false val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions) @@ -291,7 +297,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl UserGroupInformation.getCurrentUser().addCredentials(credentials) return localResources } - + def setupLaunchEnv( localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { @@ -304,16 +310,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir - // set the environment variables to be passed on to the Workers + // Set the environment variables to be passed on to the Workers. distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) - // allow users to specify some environment variables + // Allow users to specify some environment variables. Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) - // Add each SPARK-* key to the environment + // Add each SPARK-* key to the environment. System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } - return env + env } def userArgsToString(clientArgs: ClientArguments): String = { @@ -323,13 +329,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl for (arg <- args){ retval.append(prefix).append(" '").append(arg).append("' ") } - retval.toString } - def createContainerLaunchContext(newApp: GetNewApplicationResponse, - localResources: HashMap[String, LocalResource], - env: HashMap[String, String]): ContainerLaunchContext = { + def createContainerLaunchContext( + newApp: GetNewApplicationResponse, + localResources: HashMap[String, LocalResource], + env: HashMap[String, String]): ContainerLaunchContext = { logInfo("Setting up container launch context") val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) amContainer.setLocalResources(localResources) @@ -337,8 +343,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() + // TODO(harvey): This can probably be a val. var amMemory = ((args.amMemory / minResMemory) * minResMemory) + - (if (0 != (args.amMemory % minResMemory)) minResMemory else 0) - YarnAllocationHandler.MEMORY_OVERHEAD + ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - + YarnAllocationHandler.MEMORY_OVERHEAD) // Extra options for the JVM var JAVA_OPTS = "" @@ -349,13 +357,18 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " - // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out. - // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same - // node, spark gc effects all other containers performance (which can also be other spark containers) - // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is - // limited to subset of cores on a node. - if (env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))) { - // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tenant machines + // Commenting it out for now - so that people can refer to the properties if required. Remove + // it once cpuset version is pushed out. The context is, default gc for server class machines + // end up using all cores to do gc - hence if there are multiple containers in same node, + // spark gc effects all other containers performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in + // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset + // of cores on a node. + val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && + java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC")) + if (useConcurrentAndIncrementalGC) { + // In our expts, using (default) throughput collector has severe perf ramnifications in + // multi-tenant machines JAVA_OPTS += " -XX:+UseConcMarkSweepGC " JAVA_OPTS += " -XX:+CMSIncrementalMode " JAVA_OPTS += " -XX:+CMSIncrementalPacing " @@ -388,28 +401,28 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") logInfo("Command for the ApplicationMaster: " + commands(0)) amContainer.setCommands(commands) - + val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] - // Memory for the ApplicationMaster + // Memory for the ApplicationMaster. capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) amContainer.setResource(capability) - // Setup security tokens + // Setup security tokens. val dob = new DataOutputBuffer() credentials.writeTokenStorageToStream(dob) amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) - return amContainer + amContainer } - + def submitApp(appContext: ApplicationSubmissionContext) = { - // Submit the application to the applications manager + // Submit the application to the applications manager. logInfo("Submitting application to ASM") super.submitApplication(appContext) } - + def monitorApplication(appId: ApplicationId): Boolean = { - while(true) { + while (true) { Thread.sleep(1000) val report = super.getApplicationReport(appId) @@ -427,16 +440,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + "\t appUser: " + report.getUser() ) - + val state = report.getYarnApplicationState() val dsStatus = report.getFinalApplicationStatus() if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { - return true + return true } } - return true + true } } @@ -469,7 +482,7 @@ object Client { Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() + Path.SEPARATOR + LOG4J_PROP) } - // normally the users app.jar is last in case conflicts with spark jars + // Normally the users app.jar is last in case conflicts with spark jars val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false") .toBoolean if (userClasspathFirst) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala index a4d6e1d87d127..6a90cc51cfbaf 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -21,52 +21,59 @@ import java.net.URI import java.nio.ByteBuffer import java.security.PrivilegedExceptionAction +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment - -import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap import org.apache.spark.Logging -class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String, - slaveId: String, hostname: String, workerMemory: Int, workerCores: Int) - extends Runnable with Logging { - + +class WorkerRunnable( + container: Container, + conf: Configuration, + masterAddress: String, + slaveId: String, + hostname: String, + workerMemory: Int, + workerCores: Int) + extends Runnable with Logging { + var rpc: YarnRPC = YarnRPC.create(conf) var cm: ContainerManager = null val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - + def run = { logInfo("Starting Worker Container") cm = connectToCM startContainer } - + def startContainer = { logInfo("Setting up ContainerLaunchContext") - + val ctx = Records.newRecord(classOf[ContainerLaunchContext]) .asInstanceOf[ContainerLaunchContext] - + ctx.setContainerId(container.getId()) ctx.setResource(container.getResource()) val localResources = prepareLocalResources ctx.setLocalResources(localResources) - + val env = prepareEnvironment ctx.setEnvironment(env) - + // Extra options for the JVM var JAVA_OPTS = "" // Set the JVM memory @@ -79,17 +86,21 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " - - // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out. - // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same - // node, spark gc effects all other containers performance (which can also be other spark containers) - // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is - // limited to subset of cores on a node. + // Commenting it out for now - so that people can refer to the properties if required. Remove + // it once cpuset version is pushed out. + // The context is, default gc for server class machines end up using all cores to do gc - hence + // if there are multiple containers in same node, spark gc effects all other containers + // performance (which can also be other spark containers) + // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in + // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset + // of cores on a node. /* else { // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont want to mess with it. - // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tennent machines + // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont + // want to mess with it. + // In our expts, using (default) throughput collector has severe perf ramnifications in + // multi-tennent machines // The options are based on // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline JAVA_OPTS += " -XX:+UseConcMarkSweepGC " @@ -116,8 +127,10 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S val commands = List[String](javaCommand + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. - // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state. - // TODO: If the OOM is not recoverable by rescheduling it on different node, then do 'something' to fail job ... akin to blacklisting trackers in mapred ? + // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in + // an inconsistent state. + // TODO: If the OOM is not recoverable by rescheduling it on different node, then do + // 'something' to fail job ... akin to blacklisting trackers in mapred ? " -XX:OnOutOfMemoryError='kill %p' " + JAVA_OPTS + " org.apache.spark.executor.CoarseGrainedExecutorBackend " + @@ -129,7 +142,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") logInfo("Setting up worker with commands: " + commands) ctx.setCommands(commands) - + // Send the start request to the ContainerManager val startReq = Records.newRecord(classOf[StartContainerRequest]) .asInstanceOf[StartContainerRequest] @@ -137,7 +150,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S cm.startContainer(startReq) } - private def setupDistributedCache(file: String, + private def setupDistributedCache( + file: String, rtype: LocalResourceType, localResources: HashMap[String, LocalResource], timestamp: String, @@ -152,12 +166,11 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S amJarRsrc.setSize(size.toLong) localResources(uri.getFragment()) = amJarRsrc } - - + def prepareLocalResources: HashMap[String, LocalResource] = { logInfo("Preparing Local resources") val localResources = HashMap[String, LocalResource]() - + if (System.getenv("SPARK_YARN_CACHE_FILES") != null) { val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',') val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',') @@ -179,30 +192,30 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S timeStamps(i), fileSizes(i), visibilities(i)) } } - + logInfo("Prepared Local resources " + localResources) return localResources } - + def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env) - // allow users to specify some environment variables + // Allow users to specify some environment variables Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } return env } - + def connectToCM: ContainerManager = { val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort() val cmAddress = NetUtils.createSocketAddr(cmHostPortStr) logInfo("Connecting to ContainerManager at " + cmHostPortStr) - // use doAs and remoteUser here so we can add the container token and not - // pollute the current users credentials with all of the individual container tokens + // Use doAs and remoteUser here so we can add the container token and not pollute the current + // users credentials with all of the individual container tokens val user = UserGroupInformation.createRemoteUser(container.getId().toString()) val containerToken = container.getContainerToken() if (containerToken != null) { @@ -218,5 +231,5 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S }) proxy } - + } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 507a0743fd77a..f15f3c7c1195c 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -17,55 +17,70 @@ package org.apache.spark.deploy.yarn +import java.lang.{Boolean => JBoolean} +import java.util.{Collections, Set => JSet} +import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} + import org.apache.spark.Logging -import org.apache.spark.util.Utils import org.apache.spark.scheduler.SplitInfo -import scala.collection -import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId, ContainerId, Priority, Resource, ResourceRequest, ContainerStatus, Container} import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend} +import org.apache.spark.util.Utils + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.AMRMProtocol +import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId} +import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus} +import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest} import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} import org.apache.hadoop.yarn.util.{RackResolver, Records} -import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} -import java.util.concurrent.atomic.AtomicInteger -import org.apache.hadoop.yarn.api.AMRMProtocol -import collection.JavaConversions._ -import collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.hadoop.conf.Configuration -import java.util.{Collections, Set => JSet} -import java.lang.{Boolean => JBoolean} + object AllocationType extends Enumeration ("HOST", "RACK", "ANY") { type AllocationType = Value val HOST, RACK, ANY = Value } -// too many params ? refactor it 'somehow' ? -// needs to be mt-safe -// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive : should make it -// more proactive and decoupled. +// TODO: +// Too many params. +// Needs to be mt-safe +// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should +// make it more proactive and decoupled. + // Note that right now, we assume all node asks as uniform in terms of capabilities and priority -// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for more info -// on how we are requesting for containers. -private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceManager: AMRMProtocol, - val appAttemptId: ApplicationAttemptId, - val maxWorkers: Int, val workerMemory: Int, val workerCores: Int, - val preferredHostToCount: Map[String, Int], - val preferredRackToCount: Map[String, Int]) +// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for +// more info on how we are requesting for containers. +private[yarn] class YarnAllocationHandler( + val conf: Configuration, + val resourceManager: AMRMProtocol, + val appAttemptId: ApplicationAttemptId, + val maxWorkers: Int, + val workerMemory: Int, + val workerCores: Int, + val preferredHostToCount: Map[String, Int], + val preferredRackToCount: Map[String, Int]) extends Logging { - - // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set - // allocatedContainerToHostMap: container to host mapping - private val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]() + // allocatedContainerToHostMap: container to host mapping. + private val allocatedHostToContainersMap = + new HashMap[String, collection.mutable.Set[ContainerId]]() + private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() - // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an allocated node) - // As with the two data structures above, tightly coupled with them, and to be locked on allocatedHostToContainersMap + + // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an + // allocated node) + // As with the two data structures above, tightly coupled with them, and to be locked on + // allocatedHostToContainersMap private val allocatedRackCount = new HashMap[String, Int]() - // containers which have been released. + // Containers which have been released. private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]() - // containers to be released in next request to RM + // Containers to be released in next request to RM private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] private val numWorkersRunning = new AtomicInteger() @@ -83,23 +98,31 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM } def allocateContainers(workersToRequest: Int) { - // We need to send the request only once from what I understand ... but for now, not modifying this much. + // We need to send the request only once from what I understand ... but for now, not modifying + // this much. // Keep polling the Resource Manager for containers val amResp = allocateWorkerResources(workersToRequest).getAMResponse val _allocatedContainers = amResp.getAllocatedContainers() - if (_allocatedContainers.size > 0) { - - logDebug("Allocated " + _allocatedContainers.size + " containers, current count " + - numWorkersRunning.get() + ", to-be-released " + releasedContainerList + - ", pendingReleaseContainers : " + pendingReleaseContainers) - logDebug("Cluster Resources: " + amResp.getAvailableResources) + if (_allocatedContainers.size > 0) { + logDebug(""" + Allocated containers: %d + Current worker count: %d + Containers released: %s + Containers to be released: %s + Cluster resources: %s + """.format( + _allocatedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers, + amResp.getAvailableResources)) val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() - // ignore if not satisfying constraints { + // Ignore if not satisfying constraints { for (container <- _allocatedContainers) { if (isResourceConstraintSatisfied(container)) { // allocatedContainers += container @@ -113,8 +136,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM else releasedContainerList.add(container.getId()) } - // Find the appropriate containers to use - // Slightly non trivial groupBy I guess ... + // Find the appropriate containers to use. Slightly non trivial groupBy ... val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() @@ -134,21 +156,22 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM remainingContainers = null } else if (requiredHostCount > 0) { - // container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - requiredHostCount) - // and rest as remainingContainer - val (dataLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredHostCount) + // Container list has more containers than we need for data locality. + // Split into two : data local container count of (remainingContainers.size - + // requiredHostCount) and rest as remainingContainer + val (dataLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredHostCount) dataLocalContainers.put(candidateHost, dataLocal) // remainingContainers = remaining // yarn has nasty habit of allocating a tonne of containers on a host - discourage this : - // add remaining to release list. If we have insufficient containers, next allocation cycle - // will reallocate (but wont treat it as data local) + // add remaining to release list. If we have insufficient containers, next allocation + // cycle will reallocate (but wont treat it as data local) for (container <- remaining) releasedContainerList.add(container.getId()) remainingContainers = null } - // now rack local + // Now rack local if (remainingContainers != null){ val rack = YarnAllocationHandler.lookupRack(conf, candidateHost) @@ -161,15 +184,17 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM if (requiredRackCount >= remainingContainers.size){ // Add all to dataLocalContainers dataLocalContainers.put(rack, remainingContainers) - // all consumed + // All consumed remainingContainers = null } else if (requiredRackCount > 0) { // container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - requiredRackCount) - // and rest as remainingContainer - val (rackLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredRackCount) - val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, new ArrayBuffer[Container]()) + // Split into two : data local container count of (remainingContainers.size - + // requiredRackCount) and rest as remainingContainer + val (rackLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredRackCount) + val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, + new ArrayBuffer[Container]()) existingRackLocal ++= rackLocal remainingContainers = remaining @@ -185,8 +210,8 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM // Now that we have split the containers into various groups, go through them in order : // first host local, then rack local and then off rack (everything else). - // Note that the list we create below tries to ensure that not all containers end up within a host - // if there are sufficiently large number of hosts/containers. + // Note that the list we create below tries to ensure that not all containers end up within a + // host if there are sufficiently large number of hosts/containers. val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size) allocatedContainers ++= ClusterScheduler.prioritizeContainers(dataLocalContainers) @@ -199,33 +224,39 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM val workerHostname = container.getNodeId.getHost val containerId = container.getId - assert (container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) + assert( + container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) if (numWorkersRunningNow > maxWorkers) { - logInfo("Ignoring container " + containerId + " at host " + workerHostname + - " .. we already have required number of containers") + logInfo("""Ignoring container %s at host %s, since we already have the required number of + containers for it.""".format(containerId, workerHostname)) releasedContainerList.add(containerId) // reset counter back to old value. numWorkersRunning.decrementAndGet() } else { - // deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter) + // Deallocate + allocate can result in reusing id's wrongly - so use a different counter + // (workerIdCounter) val workerId = workerIdCounter.incrementAndGet().toString val driverUrl = "akka://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + workerHostname) - // just to be safe, simply remove it from pendingReleaseContainers. Should not be there, but .. + // Just to be safe, simply remove it from pendingReleaseContainers. + // Should not be there, but .. pendingReleaseContainers.remove(containerId) val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, new HashSet[ContainerId]()) + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, + new HashSet[ContainerId]()) containerSet += containerId allocatedContainerToHostMap.put(containerId, workerHostname) - if (rack != null) allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) + if (rack != null) { + allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) + } } new Thread( @@ -234,17 +265,23 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM ).start() } } - logDebug("After allocated " + allocatedContainers.size + " containers (orig : " + - _allocatedContainers.size + "), current count " + numWorkersRunning.get() + - ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers) + logDebug(""" + Finished processing %d containers. + Current number of workers running: %d, + releasedContainerList: %s, + pendingReleaseContainers: %s + """.format( + allocatedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers)) } val completedContainers = amResp.getCompletedContainersStatuses() if (completedContainers.size > 0){ - logDebug("Completed " + completedContainers.size + " containers, current count " + numWorkersRunning.get() + - ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers) - + logDebug("Completed %d containers, to-be-released: %s".format( + completedContainers.size, releasedContainerList)) for (completedContainer <- completedContainers){ val containerId = completedContainer.getContainerId @@ -253,16 +290,17 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM pendingReleaseContainers.remove(containerId) } else { - // simply decrement count - next iteration of ReporterThread will take care of allocating ! + // Simply decrement count - next iteration of ReporterThread will take care of allocating. numWorkersRunning.decrementAndGet() - logInfo("Container completed not by us ? nodeId: " + containerId + ", state " + completedContainer.getState + - " httpaddress: " + completedContainer.getDiagnostics + " exit status: " + completedContainer.getExitStatus()) - + logInfo("Completed container %s (state: %s, exit status: %s)".format( + containerId, + completedContainer.getState, + completedContainer.getExitStatus())) // Hadoop 2.2.X added a ContainerExitStatus we should switch to use // there are some exit status' we shouldn't necessarily count against us, but for // now I think its ok as none of the containers are expected to exit if (completedContainer.getExitStatus() != 0) { - logInfo("Container marked as failed: " + containerId) + logInfo("Container marked as failed: " + containerId) numWorkersFailed.incrementAndGet() } } @@ -281,7 +319,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM allocatedContainerToHostMap -= containerId - // doing this within locked context, sigh ... move to outside ? + // Doing this within locked context, sigh ... move to outside ? val rack = YarnAllocationHandler.lookupRack(conf, host) if (rack != null) { val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 @@ -291,9 +329,16 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM } } } - logDebug("After completed " + completedContainers.size + " containers, current count " + - numWorkersRunning.get() + ", to-be-released " + releasedContainerList + - ", pendingReleaseContainers : " + pendingReleaseContainers) + logDebug(""" + Finished processing %d completed containers. + Current number of workers running: %d, + releasedContainerList: %s, + pendingReleaseContainers: %s + """.format( + completedContainers.size, + numWorkersRunning.get(), + releasedContainerList, + pendingReleaseContainers)) } } @@ -347,7 +392,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM // default. if (numWorkers <= 0 || preferredHostToCount.isEmpty) { - logDebug("numWorkers: " + numWorkers + ", host preferences ? " + preferredHostToCount.isEmpty) + logDebug("numWorkers: " + numWorkers + ", host preferences: " + preferredHostToCount.isEmpty) resourceRequests = List( createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)) } @@ -360,17 +405,24 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) if (requiredCount > 0) { - hostContainerRequests += - createResourceRequest(AllocationType.HOST, candidateHost, requiredCount, YarnAllocationHandler.PRIORITY) + hostContainerRequests += createResourceRequest( + AllocationType.HOST, + candidateHost, + requiredCount, + YarnAllocationHandler.PRIORITY) } } - val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(hostContainerRequests.toList) + val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests( + hostContainerRequests.toList) - val anyContainerRequests: ResourceRequest = - createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY) + val anyContainerRequests: ResourceRequest = createResourceRequest( + AllocationType.ANY, + resource = null, + numWorkers, + YarnAllocationHandler.PRIORITY) - val containerRequests: ArrayBuffer[ResourceRequest] = - new ArrayBuffer[ResourceRequest](hostContainerRequests.size() + rackContainerRequests.size() + 1) + val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( + hostContainerRequests.size + rackContainerRequests.size + 1) containerRequests ++= hostContainerRequests containerRequests ++= rackContainerRequests @@ -389,52 +441,59 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM req.addAllReleases(releasedContainerList) if (numWorkers > 0) { - logInfo("Allocating " + numWorkers + " worker containers with " + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + " of memory each.") + logInfo("Allocating %d worker containers with %d of memory each.".format(numWorkers, + workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) } else { logDebug("Empty allocation req .. release : " + releasedContainerList) } - for (req <- resourceRequests) { - logInfo("rsrcRequest ... host : " + req.getHostName + ", numContainers : " + req.getNumContainers + - ", p = " + req.getPriority().getPriority + ", capability: " + req.getCapability) + for (request <- resourceRequests) { + logInfo("ResourceRequest (host : %s, num containers: %d, priority = %s , capability : %s)". + format( + request.getHostName, + request.getNumContainers, + request.getPriority, + request.getCapability)) } resourceManager.allocate(req) } - private def createResourceRequest(requestType: AllocationType.AllocationType, - resource:String, numWorkers: Int, priority: Int): ResourceRequest = { + private def createResourceRequest( + requestType: AllocationType.AllocationType, + resource:String, + numWorkers: Int, + priority: Int): ResourceRequest = { // If hostname specified, we need atleast two requests - node local and rack local. // There must be a third request - which is ANY : that will be specially handled. requestType match { case AllocationType.HOST => { - assert (YarnAllocationHandler.ANY_HOST != resource) - + assert(YarnAllocationHandler.ANY_HOST != resource) val hostname = resource val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority) - // add to host->rack mapping + // Add to host->rack mapping YarnAllocationHandler.populateRackInfo(conf, hostname) nodeLocal } - case AllocationType.RACK => { val rack = resource createResourceRequestImpl(rack, numWorkers, priority) } - - case AllocationType.ANY => { - createResourceRequestImpl(YarnAllocationHandler.ANY_HOST, numWorkers, priority) - } - - case _ => throw new IllegalArgumentException("Unexpected/unsupported request type .. " + requestType) + case AllocationType.ANY => createResourceRequestImpl( + YarnAllocationHandler.ANY_HOST, numWorkers, priority) + case _ => throw new IllegalArgumentException( + "Unexpected/unsupported request type: " + requestType) } } - private def createResourceRequestImpl(hostname:String, numWorkers: Int, priority: Int): ResourceRequest = { + private def createResourceRequestImpl( + hostname:String, + numWorkers: Int, + priority: Int): ResourceRequest = { val rsrcRequest = Records.newRecord(classOf[ResourceRequest]) val memCapability = Records.newRecord(classOf[Resource]) @@ -455,11 +514,11 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM def createReleasedContainerList(): ArrayBuffer[ContainerId] = { val retval = new ArrayBuffer[ContainerId](1) - // iterator on COW list ... + // Iterator on COW list ... for (container <- releasedContainerList.iterator()){ retval += container } - // remove from the original list. + // Remove from the original list. if (! retval.isEmpty) { releasedContainerList.removeAll(retval) for (v <- retval) pendingReleaseContainers.put(v, true) @@ -474,14 +533,14 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM object YarnAllocationHandler { val ANY_HOST = "*" - // all requests are issued with same priority : we do not (yet) have any distinction between + // All requests are issued with same priority : we do not (yet) have any distinction between // request types (like map/reduce in hadoop for example) val PRIORITY = 1 // Additional memory overhead - in mb val MEMORY_OVERHEAD = 384 - // host to rack map - saved from allocation requests + // Host to rack map - saved from allocation requests // We are expecting this not to change. // Note that it is possible for this to change : and RM will indicate that to us via update // response to allocate. But we are punting on handling that for now. @@ -489,38 +548,69 @@ object YarnAllocationHandler { private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() - def newAllocator(conf: Configuration, - resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments): YarnAllocationHandler = { - - new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers, - args.workerMemory, args.workerCores, Map[String, Int](), Map[String, Int]()) + def newAllocator( + conf: Configuration, + resourceManager: AMRMProtocol, + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments): YarnAllocationHandler = { + + new YarnAllocationHandler( + conf, + resourceManager, + appAttemptId, + args.numWorkers, + args.workerMemory, + args.workerCores, + Map[String, Int](), + Map[String, Int]()) } - def newAllocator(conf: Configuration, - resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { + def newAllocator( + conf: Configuration, + resourceManager: AMRMProtocol, + appAttemptId: ApplicationAttemptId, + args: ApplicationMasterArguments, + map: collection.Map[String, + collection.Set[SplitInfo]]): YarnAllocationHandler = { val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) - - new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers, - args.workerMemory, args.workerCores, hostToCount, rackToCount) + new YarnAllocationHandler( + conf, + resourceManager, + appAttemptId, + args.numWorkers, + args.workerMemory, + args.workerCores, + hostToCount, + rackToCount) } - def newAllocator(conf: Configuration, - resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, - maxWorkers: Int, workerMemory: Int, workerCores: Int, - map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { + def newAllocator( + conf: Configuration, + resourceManager: AMRMProtocol, + appAttemptId: ApplicationAttemptId, + maxWorkers: Int, + workerMemory: Int, + workerCores: Int, + map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = { val (hostToCount, rackToCount) = generateNodeToWeight(conf, map) - new YarnAllocationHandler(conf, resourceManager, appAttemptId, maxWorkers, - workerMemory, workerCores, hostToCount, rackToCount) + new YarnAllocationHandler( + conf, + resourceManager, + appAttemptId, + maxWorkers, + workerMemory, + workerCores, + hostToCount, + rackToCount) } // A simple method to copy the split info map. - private def generateNodeToWeight(conf: Configuration, input: collection.Map[String, collection.Set[SplitInfo]]) : + private def generateNodeToWeight( + conf: Configuration, + input: collection.Map[String, collection.Set[SplitInfo]]) : // host to count, rack to count (Map[String, Int], Map[String, Int]) = { @@ -544,7 +634,7 @@ object YarnAllocationHandler { } def lookupRack(conf: Configuration, host: String): String = { - if (! hostToRack.contains(host)) populateRackInfo(conf, host) + if (!hostToRack.contains(host)) populateRackInfo(conf, host) hostToRack.get(host) } @@ -567,10 +657,12 @@ object YarnAllocationHandler { val rack = rackInfo.getNetworkLocation hostToRack.put(hostname, rack) if (! rackToHostSet.containsKey(rack)) { - rackToHostSet.putIfAbsent(rack, Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]())) + rackToHostSet.putIfAbsent(rack, + Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]())) } rackToHostSet.get(rack).add(hostname) + // TODO(harvey): Figure out this comment... // Since RackResolver caches, we are disabling this for now ... } /* else { // right ? Else we will keep calling rack resolver in case we cant resolve rack info ... From 07470d130bba48e90e86f9cffb4b5af271a394ac Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 00:15:00 -0800 Subject: [PATCH 37/90] Small fix for Harvey's patch --- new-yarn/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml index 8a065c6d7d1d7..d9168e33f6084 100644 --- a/new-yarn/pom.xml +++ b/new-yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.9.0-incubating-SNAPSHOT + 0.8.1-incubating-SNAPSHOT ../pom.xml From 264231293915480d63af7fc71b1c822692c36c49 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 6 Dec 2013 17:29:03 -0800 Subject: [PATCH 38/90] Merge pull request #234 from alig/master Updated documentation about the YARN v2.2 build process (cherry picked from commit 241336add5be07fca5ff6c17eed368df7d0c3e3c) Signed-off-by: Patrick Wendell --- docs/building-with-maven.md | 4 ++++ docs/cluster-overview.md | 2 +- docs/index.md | 6 ++++-- docs/running-on-yarn.md | 8 ++++++++ 4 files changed, 17 insertions(+), 3 deletions(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 19c01e179f35f..a50878630d153 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -45,6 +45,10 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with # Cloudera CDH 4.2.0 with MapReduce v2 $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package +Hadoop versions 2.2.x and newer can be built by setting the ```new-yarn``` and the ```yarn.version``` as follows: + mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn + +The build process handles Hadoop 2.2.x as a special case that uses the directory ```new-yarn```, which supports the new YARN API. Furthermore, for this version, the build depends on artifacts published by the spark-project to enable Akka 2.0.5 to work with protobuf 2.5. ## Spark Tests in Maven ## diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 5927f736f3579..e16703292cc22 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -45,7 +45,7 @@ The system currently supports three cluster managers: easy to set up a cluster. * [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce and service applications. -* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.0. +* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2. In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone cluster on Amazon EC2. diff --git a/docs/index.md b/docs/index.md index bd386a8a8fdb6..bbb27338bc054 100644 --- a/docs/index.md +++ b/docs/index.md @@ -56,14 +56,16 @@ Hadoop, you must build Spark against the same version that your cluster uses. By default, Spark links to Hadoop 1.0.4. You can change this by setting the `SPARK_HADOOP_VERSION` variable when compiling: - SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly + SPARK_HADOOP_VERSION=2.2.0 sbt/sbt assembly In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set `SPARK_YARN` to `true`: SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly -(Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`.) +Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`. + +For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.md). This is needed because Hadoop 2.2 has non backwards compatible API changes. # Where to Go from Here diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 68fd6c2ab1db2..ae65127759c28 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -17,6 +17,7 @@ This can be built by setting the Hadoop version and `SPARK_YARN` environment var The assembled JAR will be something like this: `./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. +The build process now also supports new YARN versions (2.2.x). See below. # Preparations @@ -111,9 +112,16 @@ For example: SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ MASTER=yarn-client ./spark-shell +# Building Spark for Hadoop/YARN 2.2.x + +Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5) that is not compatible with the Akka version (2.0.5) that Spark uses. Therefore, if the Hadoop version (e.g. set through ```SPARK_HADOOP_VERSION```) starts with 2.2.0 or higher then the build process will depend on Akka artifacts distributed by the Spark project compatible with Protobuf 2.5. Furthermore, the build process then uses the directory ```new-yarn``` (instead of ```yarn```), which supports the new YARN API. The build process should seamlessly work out of the box. + +See [Building Spark with Maven](building-with-maven.md) for instructions on how to build Spark using the Maven process. + # Important Notes - We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed. - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored. - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN. - The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. +- YARN 2.2.x users cannot simply depend on the Spark packages without building Spark, as the published Spark artifacts are compiled to work with the pre 2.2 API. Those users must build Spark and publish it locally. \ No newline at end of file From 4a6aae3bb0936dd2b6592e787ff7ffe9c5236251 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 6 Dec 2013 20:14:56 -0800 Subject: [PATCH 39/90] Merge pull request #235 from pwendell/master Minor doc fixes and updating README (cherry picked from commit e5d5728b72e58046cc175ab06b5f1c7be4957711) Signed-off-by: Patrick Wendell --- README.md | 7 ++++++- docs/building-with-maven.md | 4 +++- docs/index.md | 2 +- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 5875505ec1a9a..37d9c0ffe7b23 100644 --- a/README.md +++ b/README.md @@ -55,7 +55,7 @@ versions without YARN, use: # Cloudera CDH 4.2.0 with MapReduce v1 $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions +For Apache Hadoop 2.0.X, 2.1.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, also set `SPARK_YARN=true`: # Apache Hadoop 2.0.5-alpha @@ -64,6 +64,11 @@ with YARN, also set `SPARK_YARN=true`: # Cloudera CDH 4.2.0 with MapReduce v2 $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly +When building for Hadoop 2.2.X and newer, you'll need to include the additional `new-yarn` profile: + + # Apache Hadoop 2.2.X and newer + $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn + For convenience, these variables may also be set through the `conf/spark-env.sh` file described below. diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index a50878630d153..c709001632261 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -46,7 +46,9 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package Hadoop versions 2.2.x and newer can be built by setting the ```new-yarn``` and the ```yarn.version``` as follows: - mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn + + # Apache Hadoop 2.2.X and newer + $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn The build process handles Hadoop 2.2.x as a special case that uses the directory ```new-yarn```, which supports the new YARN API. Furthermore, for this version, the build depends on artifacts published by the spark-project to enable Akka 2.0.5 to work with protobuf 2.5. diff --git a/docs/index.md b/docs/index.md index bbb27338bc054..45616f78f99fd 100644 --- a/docs/index.md +++ b/docs/index.md @@ -65,7 +65,7 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`. -For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.md). This is needed because Hadoop 2.2 has non backwards compatible API changes. +For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.html). This is needed because Hadoop 2.2 has non backwards compatible API changes. # Where to Go from Here From 80cc4ff94f1cbf53314ee6117733864b690e0249 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 6 Dec 2013 20:29:45 -0800 Subject: [PATCH 40/90] Merge pull request #237 from pwendell/formatting-fix Formatting fix This is a single-line change. The diff appears larger here due to github being out of sync. (cherry picked from commit 10c3c0c6524d0cf6c59b6f2227bf316cdeb7d06c) Signed-off-by: Patrick Wendell --- docs/configuration.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 22abe1c5a9943..0edbac294f462 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -334,7 +334,6 @@ Apart from these, the following properties are also available, and may be useful If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance if you run shuffles with large numbers of reduce tasks. - spark.speculation false From cfca70e4b285a321e9a4543f3965430e6d72b2fd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 6 Dec 2013 20:16:15 -0800 Subject: [PATCH 41/90] Merge pull request #236 from pwendell/shuffle-docs Adding disclaimer for shuffle file consolidation (cherry picked from commit 1b38f5f2774982d524742e987b6cef26ccaae676) Signed-off-by: Patrick Wendell --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 0edbac294f462..e86b9ea645b26 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -331,7 +331,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.consolidateFiles false - If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance if you run shuffles with large numbers of reduce tasks. + If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is reccomended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations. From d6e5eab2f68d260cf89fa859ad6d288b33149648 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 00:15:41 -0800 Subject: [PATCH 42/90] typo fix --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index e86b9ea645b26..2accfe84c606e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -331,7 +331,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.consolidateFiles false - If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is reccomended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations. + If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is recomended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations. From 92597c0c1cffd0d4a9a9d5795c49b9dfde59c4fd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 11:56:16 -0800 Subject: [PATCH 43/90] Merge pull request #240 from pwendell/master SPARK-917 Improve API links in nav bar (cherry picked from commit 6494d62fe40ac408b14de3f0f3de8ec896a0ae6e) Signed-off-by: Patrick Wendell --- docs/_layouts/global.html | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 0c1d657cde470..ad7969d012283 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -74,12 +74,12 @@ From 30bcd84ae9f4f41369043e885164baccc615717a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 12:06:08 -0800 Subject: [PATCH 44/90] Clean-up of changes file --- CHANGES.txt | 1290 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 1290 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 311d90526e28a..60b1e8ba78ff7 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -2,6 +2,45 @@ Spark Change Log Release 0.8.1-incubating + 92597c0 Sat Dec 7 11:58:00 2013 -0800 + Merge pull request #240 from pwendell/master + + cfca70e Sat Dec 7 01:15:20 2013 -0800 + Merge pull request #236 from pwendell/shuffle-docs + + 80cc4ff Sat Dec 7 01:15:20 2013 -0800 + Merge pull request #237 from pwendell/formatting-fix + + 4a6aae3 Sat Dec 7 01:15:20 2013 -0800 + Merge pull request #235 from pwendell/master + + 2642312 Sat Dec 7 01:15:20 2013 -0800 + Merge pull request #234 from alig/master + + 2d3eae2 Sat Dec 7 01:15:19 2013 -0800 + Merge pull request #199 from harveyfeng/yarn-2.2 + + 1e9d084 Sat Dec 7 01:15:19 2013 -0800 + Merge pull request #101 from colorant/yarn-client-scheduler + + 20d1f8b Sat Dec 7 01:15:09 2013 -0800 + Merge pull request #191 from hsaputra/removesemicolonscala + + 2b76315 Sat Dec 7 00:02:14 2013 -0800 + Merge pull request #178 from hsaputra/simplecleanupcode + + ee22be0 Fri Dec 6 23:29:38 2013 -0800 + Merge pull request #189 from tgravescs/sparkYarnErrorHandling + + d77c337 Thu Dec 5 23:30:11 2013 -0800 + Merge pull request #232 from markhamstra/FiniteWait + + 17ca8a1 Thu Dec 5 14:32:01 2013 -0800 + Merge pull request #231 from pwendell/branch-0.8 + + 47fce43 Thu Dec 5 12:33:02 2013 -0800 + Merge pull request #228 from pwendell/master + cc33f9f Wed Dec 4 15:57:47 2013 -0800 Merge pull request #227 from pwendell/master @@ -368,3 +407,1254 @@ Release 0.8.1-incubating d5a8dbf Thu Sep 26 13:09:30 2013 -0700 Merge pull request #928 from jerryshao/fairscheduler-refactor +Release 0.8.0-incubating + + 2aff798 Sun Sep 15 14:05:04 2013 -0700 + Merge pull request #933 from jey/yarn-typo-fix + [Fix typo in Maven build docs] + + dbd2c4f Sun Sep 15 13:20:41 2013 -0700 + Merge pull request #932 from pwendell/mesos-version + [Bumping Mesos version to 0.13.0] + + 9fb0b9d Sun Sep 15 13:02:53 2013 -0700 + Merge pull request #931 from pwendell/yarn-docs + [Explain yarn.version in Maven build docs] + + c4c1db2 Fri Sep 13 19:52:12 2013 -0700 + Merge pull request #929 from pwendell/master + [Use different Hadoop version for YARN artifacts.] + + a310de6 Wed Sep 11 19:36:11 2013 -0700 + Merge pull request #926 from kayousterhout/dynamic + [Changed localProperties to use ThreadLocal (not DynamicVariable).] + + 58c7d8b Wed Sep 11 17:33:42 2013 -0700 + Merge pull request #927 from benh/mesos-docs + [Updated Spark on Mesos documentation.] + + 91a59e6 Wed Sep 11 10:21:48 2013 -0700 + Merge pull request #919 from mateiz/jets3t + [Add explicit jets3t dependency, which is excluded in hadoop-client] + + b9128d3 Wed Sep 11 10:03:06 2013 -0700 + Merge pull request #922 from pwendell/port-change + [Change default port number from 3030 to 4030.] + + e07eef8 Wed Sep 11 07:35:39 2013 -0700 + Merge pull request #925 from davidmccauley/master + [SPARK-894 - Not all WebUI fields delivered VIA JSON] + + 8432f27 Tue Sep 10 23:19:53 2013 -0700 + Merge pull request #923 from haoyuan/master + [fix run-example script] + + d40f140 Tue Sep 10 23:05:29 2013 -0700 + Merge pull request #921 from pwendell/master + [Fix HDFS access bug with assembly build.] + + 0a6c051 Mon Sep 9 23:37:57 2013 -0700 + Merge pull request #918 from pwendell/branch-0.8 + [Update versions for 0.8.0 release.] + + 8c14f4b Mon Sep 9 22:07:58 2013 -0700 + Merge pull request #917 from pwendell/master + [Document libgfortran dependency for MLBase] + + c81377b Mon Sep 9 20:16:19 2013 -0700 + Merge pull request #915 from ooyala/master + [Get rid of / improve ugly NPE when Utils.deleteRecursively() fails] + + 61d2a01 Mon Sep 9 18:21:01 2013 -0700 + Merge pull request #916 from mateiz/mkdist-fix + [Fix copy issue in https://github.com/mesos/spark/pull/899] + + a85758c Mon Sep 9 13:45:40 2013 -0700 + Merge pull request #907 from stephenh/document_coalesce_shuffle + [Add better docs for coalesce.] + + 084fc36 Mon Sep 9 12:01:35 2013 -0700 + Merge pull request #912 from tgravescs/ganglia-pom + [Add metrics-ganglia to core pom file] + + 0456384 Mon Sep 9 09:57:54 2013 -0700 + Merge pull request #911 from pwendell/ganglia-sink + [Adding Manen dependency for Ganglia] + + bf984e2 Sun Sep 8 23:50:24 2013 -0700 + Merge pull request #890 from mridulm/master + [Fix hash bug] + + e9d4f44 Sun Sep 8 23:36:48 2013 -0700 + Merge pull request #909 from mateiz/exec-id-fix + [Fix an instance where full standalone mode executor IDs were passed to] + + 2447b1c Sun Sep 8 22:27:49 2013 -0700 + Merge pull request #910 from mateiz/ml-doc-tweaks + [Small tweaks to MLlib docs] + + 7d3204b Sun Sep 8 21:39:12 2013 -0700 + Merge pull request #905 from mateiz/docs2 + [Job scheduling and cluster mode docs] + + f1f8371 Sun Sep 8 21:26:11 2013 -0700 + Merge pull request #896 from atalwalkar/master + [updated content] + + f68848d Sun Sep 8 18:32:16 2013 -0700 + Merge pull request #906 from pwendell/ganglia-sink + [Clean-up of Metrics Code/Docs and Add Ganglia Sink] + + 0b95799 Sun Sep 8 15:30:16 2013 -0700 + Merge pull request #908 from pwendell/master + [Fix target JVM version in scala build] + + 04cfb3a Sun Sep 8 10:33:20 2013 -0700 + Merge pull request #898 from ilikerps/660 + [SPARK-660: Add StorageLevel support in Python] + + 38488ac Sun Sep 8 00:28:53 2013 -0700 + Merge pull request #900 from pwendell/cdh-docs + [Provide docs to describe running on CDH/HDP cluster.] + + a8e376e Sat Sep 7 21:16:01 2013 -0700 + Merge pull request #904 from pwendell/master + [Adding Apache license to two files] + + cfde85e Sat Sep 7 13:53:08 2013 -0700 + Merge pull request #901 from ooyala/2013-09/0.8-doc-changes + [0.8 Doc changes for make-distribution.sh] + + 4a7813a Sat Sep 7 13:52:24 2013 -0700 + Merge pull request #903 from rxin/resulttask + [Fixed the bug that ResultTask was not properly deserializing outputId.] + + afe46ba Sat Sep 7 07:28:51 2013 -0700 + Merge pull request #892 from jey/fix-yarn-assembly + [YARN build fixes] + + 2eebeff Fri Sep 6 15:25:22 2013 -0700 + Merge pull request #897 from pwendell/master + [Docs describing Spark monitoring and instrumentation] + + ddcb9d3 Thu Sep 5 23:54:09 2013 -0700 + Merge pull request #895 from ilikerps/821 + [SPARK-821: Don't cache results when action run locally on driver] + + 699c331 Thu Sep 5 20:21:53 2013 -0700 + Merge pull request #891 from xiajunluan/SPARK-864 + [[SPARK-864]DAGScheduler Exception if we delete Worker and StandaloneExecutorBackend then add Worker] + + 5c7494d Wed Sep 4 22:47:03 2013 -0700 + Merge pull request #893 from ilikerps/master + [SPARK-884: Add unit test to validate Spark JSON output] + + a547866 Wed Sep 4 21:11:56 2013 -0700 + Merge pull request #894 from c0s/master + [Updating assembly README to reflect recent changes in the build.] + + 19f7027 Tue Sep 3 14:29:10 2013 -0700 + Merge pull request #878 from tgravescs/yarnUILink + [Link the Spark UI up to the Yarn UI ] + + 68df246 Tue Sep 3 13:01:17 2013 -0700 + Merge pull request #889 from alig/master + [Return the port the WebUI is bound to (useful if port 0 was used)] + + d3dd48f Mon Sep 2 16:44:54 2013 -0700 + Merge pull request #887 from mateiz/misc-fixes + [Miscellaneous fixes for 0.8] + + 636fc0c Mon Sep 2 11:20:39 2013 -0700 + Merge pull request #886 from mateiz/codec + [Fix spark.io.compression.codec and change default codec to LZF] + + d9a53b9 Sun Sep 1 22:12:30 2013 -0700 + Merge pull request #885 from mateiz/win-py + [Allow PySpark to run on Windows] + + 3c520fe Sun Sep 1 17:26:55 2013 -0700 + Merge pull request #884 from mateiz/win-fixes + [Run script fixes for Windows after package & assembly change] + + f957c26 Sun Sep 1 14:53:57 2013 -0700 + Merge pull request #882 from mateiz/package-rename + [Rename spark package to org.apache.spark] + + a30fac1 Sun Sep 1 12:27:50 2013 -0700 + Merge pull request #883 from alig/master + [Don't require the spark home environment variable to be set for standalone mode (change needed by SIMR)] + + 03cc765 Sun Sep 1 10:20:56 2013 -0700 + Merge pull request #881 from pwendell/master + [Extend QuickStart to include next steps] + + 0e9565a Sat Aug 31 18:55:41 2013 -0700 + Merge pull request #880 from mateiz/ui-tweaks + [Various UI tweaks] + + 2b29a1d Sat Aug 31 17:49:45 2013 -0700 + Merge pull request #877 from mateiz/docs + [Doc improvements for 0.8] + + 6edef9c Sat Aug 31 13:39:24 2013 -0700 + Merge pull request #861 from AndreSchumacher/pyspark_sampling_function + [Pyspark sampling function] + + fd89835 Sat Aug 31 13:18:12 2013 -0700 + Merge pull request #870 from JoshRosen/spark-885 + [Don't send SIGINT / ctrl-c to Py4J gateway subprocess] + + 618f0ec Fri Aug 30 18:17:13 2013 -0700 + Merge pull request #869 from AndreSchumacher/subtract + [PySpark: implementing subtractByKey(), subtract() and keyBy()] + + 94bb7fd Fri Aug 30 12:05:13 2013 -0700 + Merge pull request #876 from mbautin/master_hadoop_rdd_conf + [Make HadoopRDD's configuration accessible] + + 9e17e45 Fri Aug 30 00:22:53 2013 -0700 + Merge pull request #875 from shivaram/build-fix + [Fix broken build by removing addIntercept] + + 016787d Thu Aug 29 22:15:14 2013 -0700 + Merge pull request #863 from shivaram/etrain-ridge + [Adding linear regression and refactoring Ridge regression to use SGD] + + 852d810 Thu Aug 29 22:13:15 2013 -0700 + Merge pull request #819 from shivaram/sgd-cleanup + [Change SVM to use {0,1} labels] + + ca71620 Thu Aug 29 21:51:14 2013 -0700 + Merge pull request #857 from mateiz/assembly + [Change build and run instructions to use assemblies] + + 1528776 Thu Aug 29 21:30:47 2013 -0700 + Merge pull request #874 from jerryshao/fix-report-bug + [Fix removed block zero size log reporting] + + abdbacf Wed Aug 28 21:11:31 2013 -0700 + Merge pull request #871 from pwendell/expose-local + [Expose `isLocal` in SparkContext.] + + afcade3 Wed Aug 28 20:15:40 2013 -0700 + Merge pull request #873 from pwendell/master + [Hot fix for command runner] + + baa84e7 Wed Aug 28 12:44:46 2013 -0700 + Merge pull request #865 from tgravescs/fixtmpdir + [Spark on Yarn should use yarn approved directories for spark.local.dir and tmp] + + cd043cf Tue Aug 27 19:50:32 2013 -0700 + Merge pull request #867 from tgravescs/yarnenvconfigs + [Spark on Yarn allow users to specify environment variables ] + + 898da7e Mon Aug 26 20:40:49 2013 -0700 + Merge pull request #859 from ianbuss/sbt_opts + [Pass SBT_OPTS environment through to sbt_launcher] + + 17bafea Mon Aug 26 11:59:32 2013 -0700 + Merge pull request #864 from rxin/json1 + [Revert json library change] + + f9fc5c1 Sat Aug 24 15:19:56 2013 -0700 + Merge pull request #603 from pwendell/ec2-updates + [Several Improvements to EC2 Scripts] + + d282c1e Fri Aug 23 11:20:20 2013 -0700 + Merge pull request #860 from jey/sbt-ide-fixes + [Fix IDE project generation under SBT] + + 5a6ac12 Thu Aug 22 22:08:03 2013 -0700 + Merge pull request #701 from ScrapCodes/documentation-suggestions + [Documentation suggestions for spark streaming.] + + 46ea0c1 Thu Aug 22 15:57:28 2013 -0700 + Merge pull request #814 from holdenk/master + [Create less instances of the random class during ALS initialization.] + + 9ac3d62 Thu Aug 22 15:51:10 2013 -0700 + Merge pull request #856 from jey/sbt-fix-hadoop-0.23.9 + [Re-add removed dependency to fix build under Hadoop 0.23.9] + + ae8ba83 Thu Aug 22 10:14:54 2013 -0700 + Merge pull request #855 from jey/update-build-docs + [Update build docs] + + 8a36fd0 Thu Aug 22 10:13:35 2013 -0700 + Merge pull request #854 from markhamstra/pomUpdate + [Synced sbt and maven builds to use the same dependencies, etc.] + + c2d00f1 Thu Aug 22 10:13:03 2013 -0700 + Merge pull request #832 from alig/coalesce + [Coalesced RDD with locality] + + e6d66c8 Wed Aug 21 17:44:31 2013 -0700 + Merge pull request #853 from AndreSchumacher/double_rdd + [Implementing SPARK-838: Add DoubleRDDFunctions methods to PySpark] + + 2905611 Tue Aug 20 17:36:14 2013 -0700 + Merge pull request #851 from markhamstra/MutablePairTE + [Removed meaningless types] + + d61337f Tue Aug 20 10:06:06 2013 -0700 + Merge pull request #844 from markhamstra/priorityRename + [Renamed 'priority' to 'jobId' and assorted minor changes] + + 8cae72e Mon Aug 19 23:40:04 2013 -0700 + Merge pull request #828 from mateiz/sched-improvements + [Scheduler fixes and improvements] + + efeb142 Mon Aug 19 19:23:50 2013 -0700 + Merge pull request #849 from mateiz/web-fixes + [Small fixes to web UI] + + abdc1f8 Mon Aug 19 18:30:56 2013 -0700 + Merge pull request #847 from rxin/rdd + [Allow subclasses of Product2 in all key-value related classes] + + 8fa0747 Sun Aug 18 17:02:54 2013 -0700 + Merge pull request #840 from AndreSchumacher/zipegg + [Implementing SPARK-878 for PySpark: adding zip and egg files to context ...] + + 1e137a5 Sat Aug 17 22:22:32 2013 -0700 + Merge pull request #846 from rxin/rdd + [Two minor RDD refactoring] + + e89ffc7 Fri Aug 16 14:02:34 2013 -0700 + Merge pull request #839 from jegonzal/zip_partitions + [Currying RDD.zipPartitions ] + + 1fb1b09 Thu Aug 15 22:15:05 2013 -0700 + Merge pull request #841 from rxin/json + [Use the JSON formatter from Scala library and removed dependency on lift-json.] + + c69c489 Thu Aug 15 20:55:09 2013 -0700 + Merge pull request #843 from Reinvigorate/bug-879 + [fixing typo in conf/slaves] + + 230ab27 Thu Aug 15 17:45:17 2013 -0700 + Merge pull request #834 from Daemoen/master + [Updated json output to allow for display of worker state] + + 659553b Thu Aug 15 16:56:31 2013 -0700 + Merge pull request #836 from pwendell/rename + [Rename `memoryBytesToString` and `memoryMegabytesToString`] + + 28369ff Thu Aug 15 16:44:02 2013 -0700 + Merge pull request #829 from JoshRosen/pyspark-unit-tests-python-2.6 + [Fix PySpark unit tests on Python 2.6] + + 1a13460 Thu Aug 15 15:50:44 2013 -0700 + Merge pull request #833 from rxin/ui + [Various UI improvements.] + + 044a088 Wed Aug 14 20:43:49 2013 -0700 + Merge pull request #831 from rxin/scheduler + [A few small scheduler / job description changes.] + + 839f2d4 Wed Aug 14 16:17:23 2013 -0700 + Merge pull request #822 from pwendell/ui-features + [Adding GC Stats to TaskMetrics (and three small fixes)] + + 63446f9 Wed Aug 14 00:17:07 2013 -0700 + Merge pull request #826 from kayousterhout/ui_fix + [Fixed 2 bugs in executor UI (incl. SPARK-877)] + + 3f14cba Tue Aug 13 20:09:51 2013 -0700 + Merge pull request #825 from shivaram/maven-repl-fix + [Set SPARK_CLASSPATH for maven repl tests] + + 596adc6 Tue Aug 13 19:41:34 2013 -0700 + Merge pull request #824 from mateiz/mesos-0.12.1 + [Update to Mesos 0.12.1] + + d316af9 Tue Aug 13 15:31:01 2013 -0700 + Merge pull request #821 from pwendell/print-launch-command + [Print run command to stderr rather than stdout] + + 1f79d21 Tue Aug 13 15:23:54 2013 -0700 + Merge pull request #818 from kayousterhout/killed_fix + [Properly account for killed tasks.] + + 622f83c Tue Aug 13 09:58:52 2013 -0700 + Merge pull request #817 from pwendell/pr_784 + [Minor clean-up in metrics servlet code] + + a0133bf Tue Aug 13 09:28:18 2013 -0700 + Merge pull request #784 from jerryshao/dev-metrics-servlet + [Add MetricsServlet for Spark metrics system] + + e2fdac6 Mon Aug 12 21:26:59 2013 -0700 + Merge pull request #802 from stayhf/SPARK-760-Python + [Simple PageRank algorithm implementation in Python for SPARK-760] + + d3525ba Mon Aug 12 21:02:39 2013 -0700 + Merge pull request #813 from AndreSchumacher/add_files_pyspark + [Implementing SPARK-865: Add the equivalent of ADD_JARS to PySpark] + + 9e02da2 Mon Aug 12 20:22:27 2013 -0700 + Merge pull request #812 from shivaram/maven-mllib-tests + [Create SparkContext in beforeAll for MLLib tests] + + 65d0d91 Mon Aug 12 19:00:57 2013 -0700 + Merge pull request #807 from JoshRosen/guava-optional + [Change scala.Option to Guava Optional in Java APIs] + + 4346f0a Mon Aug 12 12:12:12 2013 -0700 + Merge pull request #809 from shivaram/sgd-cleanup + [Clean up scaladoc in ML Lib.] + + ea1b4ba Mon Aug 12 08:09:58 2013 -0700 + Merge pull request #806 from apivovarov/yarn-205 + [Changed yarn.version to 2.0.5 in pom.xml] + + 2a39d2c Sun Aug 11 20:35:09 2013 -0700 + Merge pull request #810 from pwendell/dead_doc_code + [Remove now dead code inside of docs] + + e5b9ed2 Sun Aug 11 17:22:47 2013 -0700 + Merge pull request #808 from pwendell/ui_compressed_bytes + [Report compressed bytes read when calculating TaskMetrics] + + 3796486 Sun Aug 11 14:51:47 2013 -0700 + Merge pull request #805 from woggle/hadoop-rdd-jobconf + [Use new Configuration() instead of slower new JobConf() in SerializableWritable] + + ff9ebfa Sun Aug 11 10:52:55 2013 -0700 + Merge pull request #762 from shivaram/sgd-cleanup + [Refactor SGD options into a new class.] + + 95c62ca Sun Aug 11 10:30:52 2013 -0700 + Merge pull request #804 from apivovarov/master + [Fixed path to JavaALS.java and JavaKMeans.java, fixed hadoop2-yarn profi...] + + 06e4f2a Sat Aug 10 18:06:23 2013 -0700 + Merge pull request #789 from MLnick/master + [Adding Scala version of PageRank example] + + 71c63de Sat Aug 10 10:21:20 2013 -0700 + Merge pull request #795 from mridulm/master + [Fix bug reported in PR 791 : a race condition in ConnectionManager and Connection] + + d17eeb9 Sat Aug 10 09:02:27 2013 -0700 + Merge pull request #785 from anfeng/master + [expose HDFS file system stats via Executor metrics] + + dce5e47 Fri Aug 9 21:53:45 2013 -0700 + Merge pull request #800 from dlyubimov/HBASE_VERSION + [Pull HBASE_VERSION in the head of sbt build] + + cd247ba Fri Aug 9 20:41:13 2013 -0700 + Merge pull request #786 from shivaram/mllib-java + [Java fixes, tests and examples for ALS, KMeans] + + b09d4b7 Fri Aug 9 13:17:08 2013 -0700 + Merge pull request #799 from woggle/sync-fix + [Remove extra synchronization in ResultTask] + + 0bc63bf Fri Aug 9 13:16:25 2013 -0700 + Merge pull request #801 from pwendell/print-launch-command + [Print launch command [Branch 0.8 version]] + + cc6b92e Fri Aug 9 13:00:33 2013 -0700 + Merge pull request #775 from pwendell/print-launch-command + [Log the launch command for Spark daemons] + + f94fc75 Fri Aug 9 10:04:03 2013 -0700 + Merge pull request #788 from shane-huang/sparkjavaopts + [For standalone mode, add worker local env setting of SPARK_JAVA_OPTS as ...] + + 63b6e02 Thu Aug 8 14:02:02 2013 -0700 + Merge pull request #797 from mateiz/chill-0.3.1 + [Update to Chill 0.3.1] + + 9955e5a Thu Aug 8 11:03:38 2013 -0700 + Merge pull request #796 from pwendell/bootstrap-design + [Bootstrap re-design] + + 5133e4b Wed Aug 7 15:50:45 2013 -0700 + Merge pull request #790 from kayousterhout/fix_throughput + [Fixed issue in UI that decreased scheduler throughput by 5x or more] + + 3c8478e Tue Aug 6 23:25:03 2013 -0700 + Merge pull request #747 from mateiz/improved-lr + [Update the Python logistic regression example] + + 6b043a6 Tue Aug 6 22:31:02 2013 -0700 + Merge pull request #724 from dlyubimov/SPARK-826 + [SPARK-826: fold(), reduce(), collect() always attempt to use java serialization] + + de6c4c9 Tue Aug 6 17:09:50 2013 -0700 + Merge pull request #787 from ash211/master + [Update spark-standalone.md] + + df4d10d Tue Aug 6 15:44:05 2013 -0700 + Merge pull request #779 from adatao/adatao-global-SparkEnv + [[HOTFIX] Extend thread safety for SparkEnv.get()] + + d2b0f0c Tue Aug 6 14:49:39 2013 -0700 + Merge pull request #770 from stayhf/SPARK-760-Java + [Simple PageRank algorithm implementation in Java for SPARK-760] + + d031f73 Mon Aug 5 22:33:00 2013 -0700 + Merge pull request #782 from WANdisco/master + [SHARK-94 Log the files computed by HadoopRDD and NewHadoopRDD] + + 1b63dea Mon Aug 5 22:21:26 2013 -0700 + Merge pull request #769 from markhamstra/NegativeCores + [SPARK-847 + SPARK-845: Zombie workers and negative cores] + + 828aff7 Mon Aug 5 21:37:33 2013 -0700 + Merge pull request #776 from gingsmith/master + [adding matrix factorization data generator] + + 8b27789 Mon Aug 5 19:14:52 2013 -0700 + Merge pull request #774 from pwendell/job-description + [Show user-defined job name in UI] + + 550b0cf Mon Aug 5 12:10:32 2013 -0700 + Merge pull request #780 from cybermaster/master + [SPARK-850] + + 22abbc1 Fri Aug 2 16:37:59 2013 -0700 + Merge pull request #772 from karenfeng/ui-843 + [Show app duration] + + 9d7dfd2 Thu Aug 1 17:41:58 2013 -0700 + Merge pull request #743 from pwendell/app-metrics + [Add application metrics to standalone master] + + 6d7afd7 Thu Aug 1 17:13:28 2013 -0700 + Merge pull request #768 from pwendell/pr-695 + [Minor clean-up of fair scheduler UI] + + 5e7b38f Thu Aug 1 14:59:33 2013 -0700 + Merge pull request #695 from xiajunluan/pool_ui + [Enhance job ui in spark ui system with adding pool information] + + 0a96493 Thu Aug 1 11:27:17 2013 -0700 + Merge pull request #760 from karenfeng/heading-update + [Clean up web UI page headers] + + cb7dd86 Thu Aug 1 11:06:10 2013 -0700 + Merge pull request #758 from pwendell/master-json + [Add JSON path to master index page] + + 58756b7 Wed Jul 31 23:45:41 2013 -0700 + Merge pull request #761 from mateiz/kmeans-generator + [Add data generator for K-means] + + ecab635 Wed Jul 31 18:16:55 2013 -0700 + Merge pull request #763 from c0s/assembly + [SPARK-842. Maven assembly is including examples libs and dependencies] + + 39c75f3 Wed Jul 31 15:52:36 2013 -0700 + Merge pull request #757 from BlackNiuza/result_task_generation + [Bug fix: SPARK-837] + + b2b86c2 Wed Jul 31 15:51:39 2013 -0700 + Merge pull request #753 from shivaram/glm-refactor + [Build changes for ML lib] + + 14bf2fe Wed Jul 31 14:18:16 2013 -0700 + Merge pull request #749 from benh/spark-executor-uri + [Added property 'spark.executor.uri' for launching on Mesos.] + + 4ba4c3f Wed Jul 31 13:14:49 2013 -0700 + Merge pull request #759 from mateiz/split-fix + [Use the Char version of split() instead of the String one in MLUtils] + + a386ced Wed Jul 31 11:22:50 2013 -0700 + Merge pull request #754 from rxin/compression + [Compression codec change] + + 0be071a Wed Jul 31 11:11:59 2013 -0700 + Merge pull request #756 from cdshines/patch-1 + [Refactored Vector.apply(length, initializer) replacing excessive code with library method] + + d4556f4 Wed Jul 31 08:48:14 2013 -0700 + Merge pull request #751 from cdshines/master + [Cleaned Partitioner & PythonPartitioner source by taking out non-related logic to Utils] + + 29b8cd3 Tue Jul 30 21:30:33 2013 -0700 + Merge pull request #755 from jerryshao/add-apache-header + [Add Apache license header to metrics system] + + e87de03 Tue Jul 30 15:00:08 2013 -0700 + Merge pull request #744 from karenfeng/bootstrap-update + [Use Bootstrap progress bars in web UI] + + ae57020 Tue Jul 30 14:56:41 2013 -0700 + Merge pull request #752 from rxin/master + [Minor mllib cleanup] + + 8aee118 Tue Jul 30 10:27:54 2013 -0700 + Merge pull request #748 from atalwalkar/master + [made SimpleUpdater consistent with other updaters] + + 468a36c Mon Jul 29 19:44:33 2013 -0700 + Merge pull request #746 from rxin/cleanup + [Internal cleanup] + + 1e1ffb1 Mon Jul 29 19:26:19 2013 -0700 + Merge pull request #745 from shivaram/loss-update-fix + [Remove duplicate loss history in Gradient Descent] + + c99b674 Mon Jul 29 16:32:55 2013 -0700 + Merge pull request #735 from karenfeng/ui-807 + [Totals for shuffle data and CPU time] + + fe7298b Mon Jul 29 14:01:00 2013 -0700 + Merge pull request #741 from pwendell/usability + [Fix two small usability issues] + + c34c0f6 Mon Jul 29 13:18:10 2013 -0700 + Merge pull request #731 from pxinghao/master + [Adding SVM and Lasso] + + f3d72ff Fri Jul 26 17:19:27 2013 -0700 + Merge pull request #739 from markhamstra/toolsPom + [Missing tools/pom.xml scalatest dependency] + + cb36677 Fri Jul 26 16:59:30 2013 -0700 + Merge pull request #738 from harsha2010/pruning + [Fix bug in Partition Pruning.] + + f3cf094 Thu Jul 25 14:53:21 2013 -0700 + Merge pull request #734 from woggle/executor-env2 + [Get more env vars from driver rather than worker] + + 51c2427 Thu Jul 25 00:03:11 2013 -0700 + Merge pull request #732 from ryanlecompte/master + [Refactor Kryo serializer support to use chill/chill-java] + + 52723b9 Wed Jul 24 14:33:02 2013 -0700 + Merge pull request #728 from jey/examples-jar-env + [Fix setting of SPARK_EXAMPLES_JAR] + + 20338c2 Wed Jul 24 14:32:24 2013 -0700 + Merge pull request #729 from karenfeng/ui-811 + [Stage Page updates] + + 5584ebc Wed Jul 24 11:46:46 2013 -0700 + Merge pull request #675 from c0s/assembly + [Building spark assembly for further consumption of the Spark project with a deployed cluster] + + a73f3ee Wed Jul 24 08:59:14 2013 -0700 + Merge pull request #671 from jerryshao/master + [Add metrics system for Spark] + + b011329 Tue Jul 23 22:50:09 2013 -0700 + Merge pull request #727 from rxin/scheduler + [Scheduler code style cleanup.] + + 876125b Tue Jul 23 22:28:21 2013 -0700 + Merge pull request #726 from rxin/spark-826 + [SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure] + + 2f1736c Tue Jul 23 15:53:30 2013 -0700 + Merge pull request #725 from karenfeng/task-start + [Creates task start events] + + 5364f64 Tue Jul 23 13:40:34 2013 -0700 + Merge pull request #723 from rxin/mllib + [Made RegressionModel serializable and added unit tests to make sure predict methods would work.] + + f369e0e Tue Jul 23 13:22:27 2013 -0700 + Merge pull request #720 from ooyala/2013-07/persistent-rdds-api + [Add a public method getCachedRdds to SparkContext] + + 401aac8 Mon Jul 22 16:57:16 2013 -0700 + Merge pull request #719 from karenfeng/ui-808 + [Creates Executors tab for Jobs UI] + + 8ae1436 Mon Jul 22 16:03:04 2013 -0700 + Merge pull request #722 from JoshRosen/spark-825 + [Fix bug: DoubleRDDFunctions.sampleStdev() computed non-sample stdev()] + + 15fb394 Sun Jul 21 10:33:38 2013 -0700 + Merge pull request #716 from c0s/webui-port + [Regression: default webui-port can't be set via command line "--webui-port" anymore] + + c40f0f2 Fri Jul 19 13:33:04 2013 -0700 + Merge pull request #711 from shivaram/ml-generators + [Move ML lib data generator files to util/] + + 413b841 Fri Jul 19 13:31:38 2013 -0700 + Merge pull request #717 from viirya/dev1 + [Do not copy local jars given to SparkContext in yarn mode] + + 0d0a47c Thu Jul 18 12:06:37 2013 -0700 + Merge pull request #710 from shivaram/ml-updates + [Updates to LogisticRegression] + + c6235b5 Thu Jul 18 11:43:48 2013 -0700 + Merge pull request #714 from adatao/master + [[BUGFIX] Fix for sbt/sbt script SPARK_HOME setting] + + 009c79e Thu Jul 18 11:41:52 2013 -0700 + Merge pull request #715 from viirya/dev1 + [fix a bug in build process that pulls in two versions of ASM.] + + 985a9e3 Wed Jul 17 22:27:19 2013 -0700 + Merge pull request #712 from stayhf/SPARK-817 + [Consistently invoke bash with /usr/bin/env bash in scripts to make code ...] + + cad48ed Tue Jul 16 21:41:28 2013 -0700 + Merge pull request #708 from ScrapCodes/dependencies-upgrade + [Dependency upgrade Akka 2.0.3 -> 2.0.5] + + 8a8a8f2 Mon Jul 15 23:09:21 2013 -0700 + Merge pull request #705 from rxin/errormessages + [Throw a more meaningful message when runJob is called to launch tasks on non-existent partitions.] + + ed8415b Mon Jul 15 16:41:04 2013 -0700 + Merge pull request #703 from karenfeng/ui-802 + [Link to job UI from standalone deploy cluster web UI] + + e3d3e6f Mon Jul 15 14:59:44 2013 -0700 + Merge pull request #702 from karenfeng/ui-fixes + [Adds app name in HTML page titles on job web UI] + + c7877d5 Sun Jul 14 12:58:13 2013 -0700 + Merge pull request #689 from BlackNiuza/application_status + [Bug fix: SPARK-796] + + 10c0593 Sun Jul 14 11:45:18 2013 -0700 + Merge pull request #699 from pwendell/ui-env + [Add `Environment` tab to SparkUI.] + + 89e8549 Sat Jul 13 16:11:08 2013 -0700 + Merge pull request #698 from Reinvigorate/sm-deps-change + [changing com.google.code.findbugs maven coordinates] + + 77c69ae Fri Jul 12 23:05:21 2013 -0700 + Merge pull request #697 from pwendell/block-locations + [Show block locations in Web UI.] + + 5a7835c Fri Jul 12 20:28:21 2013 -0700 + Merge pull request #691 from karenfeng/logpaging + [Create log pages] + + 71ccca0 Fri Jul 12 20:25:06 2013 -0700 + Merge pull request #696 from woggle/executor-env + [Pass executor env vars (e.g. SPARK_CLASSPATH) to compute-classpath.sh] + + 90fc3f3 Fri Jul 12 20:23:36 2013 -0700 + Merge pull request #692 from Reinvigorate/takeOrdered + [adding takeOrdered() to RDD] + + 018d04c Thu Jul 11 12:48:37 2013 -0700 + Merge pull request #684 from woggle/mesos-classloader + [Explicitly set class loader for MesosSchedulerDriver callbacks.] + + bc19477 Wed Jul 10 22:29:41 2013 -0700 + Merge pull request #693 from c0s/readme + [Updating README to reflect Scala 2.9.3 requirements] + + 7dcda9a Mon Jul 8 23:24:23 2013 -0700 + Merge pull request #688 from markhamstra/scalaDependencies + [Fixed SPARK-795 with explicit dependencies] + + 638927b Mon Jul 8 22:58:50 2013 -0700 + Merge pull request #683 from shivaram/sbt-test-fix + [Remove some stack traces from sbt test output] + + 3c13178 Mon Jul 8 14:50:34 2013 -0700 + Merge pull request #687 from atalwalkar/master + [Added "Labeled" to util functions for labeled data] + + 744da8e Sun Jul 7 17:42:25 2013 -0700 + Merge pull request #679 from ryanlecompte/master + [Make binSearch method tail-recursive for RidgeRegression] + + 3cc6818 Sat Jul 6 19:51:20 2013 -0700 + Merge pull request #668 from shimingfei/guava-14.0.1 + [update guava version from 11.0.1 to 14.0.1] + + 2216188 Sat Jul 6 16:18:15 2013 -0700 + Merge pull request #676 from c0s/asf-avro + [Use standard ASF published avro module instead of a proprietory built one] + + 94871e4 Sat Jul 6 15:26:19 2013 -0700 + Merge pull request #655 from tgravescs/master + [Add support for running Spark on Yarn on a secure Hadoop Cluster] + + 3f918b3 Sat Jul 6 12:45:18 2013 -0700 + Merge pull request #672 from holdenk/master + [s/ActorSystemImpl/ExtendedActorSystem/ as ActorSystemImpl results in a warning] + + 2a36e54 Sat Jul 6 12:43:21 2013 -0700 + Merge pull request #673 from xiajunluan/master + [Add config template file for fair scheduler feature] + + 7ba7fa1 Sat Jul 6 11:45:08 2013 -0700 + Merge pull request #674 from liancheng/master + [Bug fix: SPARK-789] + + f4416a1 Sat Jul 6 11:41:58 2013 -0700 + Merge pull request #681 from BlackNiuza/memory_leak + [Remove active job from idToActiveJob when job finished or aborted] + + e063e29 Fri Jul 5 21:54:52 2013 -0700 + Merge pull request #680 from tdas/master + [Fixed major performance bug in Network Receiver] + + bf1311e Fri Jul 5 17:32:44 2013 -0700 + Merge pull request #678 from mateiz/ml-examples + [Start of ML package] + + 6ad85d0 Thu Jul 4 21:32:29 2013 -0700 + Merge pull request #677 from jerryshao/fix_stage_clean + [Clean StageToInfos periodically when spark.cleaner.ttl is enabled] + + 2e32fc8 Thu Jul 4 12:18:20 2013 -0700 + Merge pull request #666 from c0s/master + [hbase dependency is missed in hadoop2-yarn profile of examples module +] + + 6d60fe5 Mon Jul 1 18:24:03 2013 -0700 + Merge pull request #666 from c0s/master + [hbase dependency is missed in hadoop2-yarn profile of examples module] + + ccfe953 Sat Jun 29 17:57:53 2013 -0700 + Merge pull request #577 from skumargithub/master + [Example of cumulative counting using updateStateByKey] + + 50ca176 Thu Jun 27 22:24:52 2013 -0700 + Merge pull request #664 from pwendell/test-fix + [Removing incorrect test statement] + + e49bc8c Wed Jun 26 11:13:33 2013 -0700 + Merge pull request #663 from stephenh/option_and_getenv + [Be cute with Option and getenv.] + + f5e32ed Tue Jun 25 09:16:57 2013 -0700 + Merge pull request #661 from mesos/streaming + [Kafka fixes and DStream.count fix for master] + + 1249e91 Mon Jun 24 21:46:33 2013 -0700 + Merge pull request #572 from Reinvigorate/sm-block-interval + [Adding spark.streaming.blockInterval property] + + cfcda95 Mon Jun 24 21:44:50 2013 -0700 + Merge pull request #571 from Reinvigorate/sm-kafka-serializers + [Surfacing decoders on KafkaInputDStream] + + 575aff6 Mon Jun 24 21:35:50 2013 -0700 + Merge pull request #567 from Reinvigorate/sm-count-fix + [Fixing count() in Spark Streaming] + + 3e61bef Sat Jun 22 16:22:47 2013 -0700 + Merge pull request #648 from shivaram/netty-dbg + [Shuffle fixes and cleanup] + + 1ef5d0d Sat Jun 22 09:35:57 2013 -0700 + Merge pull request #644 from shimingfei/joblogger + [add Joblogger to Spark (on new Spark code)] + + 7e4b266 Sat Jun 22 07:53:18 2013 -0700 + Merge pull request #563 from jey/python-optimization + [Optimize PySpark worker invocation] + + 71030ba Wed Jun 19 15:21:03 2013 -0700 + Merge pull request #654 from lyogavin/enhance_pipe + [fix typo and coding style in #638] + + 73f4c7d Tue Jun 18 04:21:17 2013 -0700 + Merge pull request #605 from esjewett/SPARK-699 + [Add hBase example (retry of pull request #596)] + + 9933836 Tue Jun 18 02:41:10 2013 -0700 + Merge pull request #647 from jerryshao/master + [Reduce ZippedPartitionsRDD's getPreferredLocations complexity from O(2^2n) to O(2^n)] + + db42451 Mon Jun 17 15:26:36 2013 -0700 + Merge pull request #643 from adatao/master + [Bug fix: Zero-length partitions result in NaN for overall mean & variance] + + e82a2ff Mon Jun 17 15:13:15 2013 -0700 + Merge pull request #653 from rxin/logging + [SPARK-781: Log the temp directory path when Spark says "Failed to create temp directory."] + + e6d1277 Mon Jun 17 12:56:25 2013 -0700 + Merge pull request #638 from lyogavin/enhance_pipe + [Enhance pipe to support more features we can do in hadoop streaming] + + f961aac Sat Jun 15 00:53:41 2013 -0700 + Merge pull request #649 from ryanlecompte/master + [Add top K method to RDD using a bounded priority queue] + + 6602d94 Fri Jun 14 10:41:31 2013 -0700 + Merge pull request #651 from rxin/groupbykey + [SPARK-772 / SPARK-774: groupByKey and cogroup should disable map side combine] + + d93851a Thu Jun 13 13:38:45 2013 -0700 + Merge pull request #645 from pwendell/compression + [Adding compression to Hadoop save functions] + + f1da591 Wed Jun 12 17:55:08 2013 -0700 + Merge pull request #646 from markhamstra/jvmArgs + [Fixed jvmArgs in maven build.] + + 0e94b73 Mon Jun 10 13:00:31 2013 -0700 + Merge pull request #625 from stephenh/fix-start-slave + [Fix start-slave not passing instance number to spark-daemon.] + + 74b91d5 Sat Jun 8 01:19:40 2013 -0700 + Merge pull request #629 from c0s/master + [Sometime Maven build runs out of PermGen space.] + + c8fc423 Fri Jun 7 22:43:18 2013 -0700 + Merge pull request #631 from jerryshao/master + [Fix block manager UI display issue when enable spark.cleaner.ttl] + + 1ae60bc Fri Jun 7 22:39:06 2013 -0700 + Merge pull request #634 from xiajunluan/master + [[Spark-753] Fix ClusterSchedulSuite unit test failed ] + + fff3728 Tue Jun 4 16:09:50 2013 -0700 + Merge pull request #640 from pwendell/timeout-update + [Fixing bug in BlockManager timeout] + + f420d4f Tue Jun 4 15:25:58 2013 -0700 + Merge pull request #639 from pwendell/timeout-update + [Bump akka and blockmanager timeouts to 60 seconds] + + 84530ba Fri May 31 17:06:13 2013 -0700 + Merge pull request #636 from rxin/unpersist + [Unpersist More block manager cleanup.] + + ef77bb7 Thu May 30 14:50:06 2013 -0700 + Merge pull request #627 from shivaram/master + [Netty and shuffle bug fixes] + + 8cb8178 Thu May 30 14:17:44 2013 -0700 + Merge pull request #628 from shivaram/zero-block-size + [Skip fetching zero-sized blocks in NIO.] + + 6ed7139 Wed May 29 10:14:22 2013 -0700 + Merge pull request #626 from stephenh/remove-add-if-no-port + [Remove unused addIfNoPort.] + + 41d230c Tue May 28 23:35:24 2013 -0700 + Merge pull request #611 from squito/classloader + [Use default classloaders for akka & deserializing task results] + + 3db1e17 Mon May 27 21:31:43 2013 -0700 + Merge pull request #620 from jerryshao/master + [Fix CheckpointRDD java.io.FileNotFoundException when calling getPreferredLocations] + + 3d4891d Sat May 25 23:38:05 2013 -0700 + Merge pull request #621 from JoshRosen/spark-613 + [Use ec2-metadata in start-slave.sh to detect if running on EC2] + + e8d4b6c Sat May 25 21:09:03 2013 -0700 + Merge pull request #529 from xiajunluan/master + [[SPARK-663]Implement Fair Scheduler in Spark Cluster Scheduler ] + + 9a3c344 Sat May 25 17:53:43 2013 -0700 + Merge pull request #624 from rxin/master + [NonJavaSerializableClass should not be Java serializable...] + + 24e41aa Fri May 24 16:48:52 2013 -0700 + Merge pull request #623 from rxin/master + [Automatically configure Netty port.] + + 69161f9 Fri May 24 14:42:13 2013 -0700 + Merge pull request #622 from rxin/master + [bug fix: Shuffle block iterator is ignoring the shuffle serializer setting.] + + dbbedfc Thu May 23 23:11:06 2013 -0700 + Merge pull request #616 from jey/maven-netty-exclusion + [Exclude old versions of Netty from Maven-based build] + + a2b0a79 Tue May 21 18:16:20 2013 -0700 + Merge pull request #619 from woggling/adjust-sampling + [Use ARRAY_SAMPLE_SIZE constant instead of hard-coded 100.0 in SizeEstimator] + + 66dac44 Tue May 21 11:41:42 2013 -0700 + Merge pull request #618 from woggling/dead-code-disttest + [DistributedSuite: remove dead code] + + 5912cc4 Fri May 17 19:58:40 2013 -0700 + Merge pull request #610 from JoshRosen/spark-747 + [Throw exception if TaskResult exceeds Akka frame size] + + 6c27c38 Thu May 16 17:33:56 2013 -0700 + Merge pull request #615 from rxin/build-fix + [Maven build fix & two other small changes] + + 2f576ab Wed May 15 18:06:24 2013 -0700 + Merge pull request #602 from rxin/shufflemerge + [Manual merge & cleanup of Shane's Shuffle Performance Optimization] + + 48c6f46 Wed May 15 10:47:19 2013 -0700 + Merge pull request #612 from ash211/patch-4 + [Docs: Mention spark shell's default for MASTER] + + 203d7b7 Wed May 15 00:47:20 2013 -0700 + Merge pull request #593 from squito/driver_ui_link + [Master UI has link to Application UI] + + 016ac86 Mon May 13 21:45:36 2013 -0700 + Merge pull request #601 from rxin/emptyrdd-master + [EmptyRDD (master branch 0.8)] + + 4b354e0 Mon May 13 17:39:19 2013 -0700 + Merge pull request #589 from mridulm/master + [Add support for instance local scheduling] + + 5dbc9b2 Sun May 12 11:03:10 2013 -0700 + Merge pull request #608 from pwendell/SPARK-738 + [SPARK-738: Spark should detect and wrap nonserializable exceptions] + + 63e1999 Fri May 10 13:54:03 2013 -0700 + Merge pull request #606 from markhamstra/foreachPartition_fix + [Actually use the cleaned closure in foreachPartition] + + 42bbe89 Wed May 8 22:30:31 2013 -0700 + Merge pull request #599 from JoshRosen/spark-670 + [Fix SPARK-670: EC2 'start' command should require -i option.] + + 0f1b7a0 Wed May 8 13:38:50 2013 -0700 + Merge pull request #596 from esjewett/master + [hBase example] + + 7af92f2 Sat May 4 22:29:17 2013 -0700 + Merge pull request #597 from JoshRosen/webui-fixes + [Two minor bug fixes for Spark Web UI] + + c74ce60 Sat May 4 22:26:35 2013 -0700 + Merge pull request #598 from rxin/blockmanager + [Fixed flaky unpersist test in DistributedSuite.] + + 3bf2c86 Fri May 3 18:27:30 2013 -0700 + Merge pull request #594 from shivaram/master + [Add zip partitions to Java API] + + 2484ad7 Fri May 3 17:08:55 2013 -0700 + Merge pull request #587 from rxin/blockmanager + [A set of shuffle map output related changes] + + 6fe9d4e Thu May 2 21:33:56 2013 -0700 + Merge pull request #592 from woggling/localdir-fix + [Don't accept generated local directory names that can't be created] + + 538ee75 Thu May 2 09:01:42 2013 -0700 + Merge pull request #581 from jerryshao/master + [fix [SPARK-740] block manage UI throws exception when enabling Spark Streaming] + + 9abcbcc Wed May 1 22:45:10 2013 -0700 + Merge pull request #591 from rxin/removerdd + [RDD.unpersist: probably the most desired feature of Spark] + + aa8fe1a Tue Apr 30 22:30:18 2013 -0700 + Merge pull request #586 from mridulm/master + [Pull request to address issues Reynold Xin reported] + + f708dda Tue Apr 30 07:51:40 2013 -0700 + Merge pull request #585 from pwendell/listener-perf + [[Fix SPARK-742] Task Metrics should not employ per-record timing by default] + + 68c07ea Sun Apr 28 20:19:33 2013 -0700 + Merge pull request #582 from shivaram/master + [Add zip partitions interface] + + f6ee9a8 Sun Apr 28 15:36:04 2013 -0700 + Merge pull request #583 from mridulm/master + [Fix issues with streaming test cases after yarn branch merge] + + cf54b82 Thu Apr 25 11:45:58 2013 -0700 + Merge pull request #580 from pwendell/quickstart + [SPARK-739 Have quickstart standlone job use README] + + 118a6c7 Wed Apr 24 08:42:30 2013 -0700 + Merge pull request #575 from mridulm/master + [Manual merge of yarn branch to trunk] + + 5d8a71c Tue Apr 16 19:48:02 2013 -0700 + Merge pull request #570 from jey/increase-codecache-size + [Increase ReservedCodeCacheSize for sbt] + + ec5e553 Sun Apr 14 08:20:13 2013 -0700 + Merge pull request #558 from ash211/patch-jackson-conflict + [Don't pull in old versions of Jackson via hadoop-core] + + c1c219e Sun Apr 14 08:11:23 2013 -0700 + Merge pull request #564 from maspotts/master + [Allow latest scala in PATH, with SCALA_HOME as override (instead of vice-versa)] + + 7c10b3e Fri Apr 12 20:55:22 2013 -0700 + Merge pull request #565 from andyk/master + [Update wording of section on RDD operations in quick start guide in docs] + + 077ae0a Thu Apr 11 19:34:14 2013 -0700 + Merge pull request #561 from ash211/patch-4 + [Add details when BlockManager heartbeats time out] + + c91ff8d Wed Apr 10 15:08:23 2013 -0700 + Merge pull request #560 from ash211/patch-3 + [Typos: cluser -> cluster] + + 7cd83bf Tue Apr 9 22:07:35 2013 -0700 + Merge pull request #559 from ash211/patch-example-whitespace + [Uniform whitespace across scala examples] + + 271a4f3 Tue Apr 9 22:04:52 2013 -0700 + Merge pull request #555 from holdenk/master + [Retry failed ssh commands in the ec2 python script.] + + 8ac9efb Tue Apr 9 13:50:50 2013 -0700 + Merge pull request #527 from Reinvigorate/sm-kafka-cleanup + [KafkaInputDStream fixes and improvements] + + eed54a2 Mon Apr 8 09:44:30 2013 -0700 + Merge pull request #553 from pwendell/akka-standalone + [SPARK-724 - Have Akka logging enabled by default for standalone daemons] + + b362df3 Sun Apr 7 17:17:52 2013 -0700 + Merge pull request #552 from MLnick/master + [Bumping version for Twitter Algebird to latest] + + 4b30190 Sun Apr 7 17:15:10 2013 -0700 + Merge pull request #554 from andyk/scala2.9.3 + [Fixes SPARK-723 - Update build to Scala 2.9.3] + + dfe98ca Tue Apr 2 19:24:12 2013 -0700 + Merge pull request #550 from erikvanoosten/master + [corrected Algebird example] + + b5d7830 Tue Apr 2 19:23:45 2013 -0700 + Merge pull request #551 from jey/python-bugfixes + [Python bugfixes] + + 2be2295 Sun Mar 31 18:09:14 2013 -0700 + Merge pull request #548 from markhamstra/getWritableClass_filter + [Fixed broken filter in getWritableClass[T]] + + 9831bc1 Fri Mar 29 22:16:22 2013 -0700 + Merge pull request #539 from cgrothaus/fix-webui-workdirpath + [Bugfix: WorkerWebUI must respect workDirPath from Worker] + + 3cc8ab6 Fri Mar 29 22:14:07 2013 -0700 + Merge pull request #541 from stephenh/shufflecoalesce + [Add a shuffle parameter to coalesce.] + + cad507a Fri Mar 29 22:13:12 2013 -0700 + Merge pull request #547 from jey/maven-streaming-tests-initialization-fix + [Move streaming test initialization into 'before' blocks] + + a98996d Fri Mar 29 22:12:15 2013 -0700 + Merge pull request #545 from ash211/patch-1 + [Don't use deprecated Application in example] + + 104c694 Fri Mar 29 22:11:50 2013 -0700 + Merge pull request #546 from ash211/patch-2 + [Update tuning.md] + + bc36ee4 Tue Mar 26 15:05:13 2013 -0700 + Merge pull request #543 from holdenk/master + [Re-enable deprecation warnings and fix deprecated warning.] + + b8949ca Sat Mar 23 07:19:34 2013 -0700 + Merge pull request #505 from stephenh/volatile + [Make Executor fields volatile since they're read from the thread pool.] + + fd53f2f Sat Mar 23 07:13:21 2013 -0700 + Merge pull request #510 from markhamstra/WithThing + [mapWith, flatMapWith and filterWith] + + 4c5efcf Wed Mar 20 19:29:23 2013 -0700 + Merge pull request #532 from andyk/master + [SPARK-715: Adds instructions for building with Maven to documentation] + + 3558849 Wed Mar 20 19:27:47 2013 -0700 + Merge pull request #538 from rxin/cogroup + [Added mapSideCombine flag to CoGroupedRDD. Added unit test for CoGroupedRDD.] + + ca4d083 Wed Mar 20 11:22:36 2013 -0700 + Merge pull request #528 from MLnick/java-examples + [[SPARK-707] Adding Java versions of Pi, LogQuery and K-Means examples] + + b812e6b Wed Mar 20 11:21:02 2013 -0700 + Merge pull request #526 from markhamstra/foldByKey + [Add foldByKey] + + 945d1e7 Tue Mar 19 21:59:06 2013 -0700 + Merge pull request #536 from sasurfer/master + [CoalescedRDD for many partitions] + + 1cbbe94 Tue Mar 19 21:34:34 2013 -0700 + Merge pull request #534 from stephenh/removetrycatch + [Remove try/catch block that can't be hit.] + + 71e53f8 Tue Mar 19 21:31:41 2013 -0700 + Merge pull request #537 from wishbear/configurableInputFormat + [call setConf from input format if it is Configurable] + + c1e9cdc Sat Mar 16 11:47:45 2013 -0700 + Merge pull request #525 from stephenh/subtractByKey + [Add PairRDDFunctions.subtractByKey.] + + cdbfd1e Fri Mar 15 15:13:28 2013 -0700 + Merge pull request #516 from squito/fix_local_metrics + [Fix local metrics] + + f9fa2ad Fri Mar 15 15:12:43 2013 -0700 + Merge pull request #530 from mbautin/master-update-log4j-and-make-compile-in-IntelliJ + [Add a log4j compile dependency to fix build in IntelliJ] + + 4032beb Wed Mar 13 19:29:46 2013 -0700 + Merge pull request #521 from stephenh/earlyclose + [Close the reader in HadoopRDD as soon as iteration end.] + + 3c97276 Wed Mar 13 19:25:08 2013 -0700 + Merge pull request #524 from andyk/master + [Fix broken link to YARN documentation] + + 1c3d981 Wed Mar 13 19:23:48 2013 -0700 + Merge pull request #517 from Reinvigorate/sm-build-fixes + [Build fixes for streaming /w SBT] + + 2d477fd Wed Mar 13 06:49:16 2013 -0700 + Merge pull request #523 from andyk/master + [Fix broken link in Quick Start] + + 00c4d23 Tue Mar 12 22:19:00 2013 -0700 + Merge pull request #518 from woggling/long-bm-sizes + [Send block sizes as longs in BlockManager updates] + + cbf8f0d Mon Mar 11 00:23:57 2013 -0700 + Merge pull request #513 from MLnick/bagel-caching + [Adds choice of persistence level to Bagel.] + + 91a9d09 Sun Mar 10 15:48:23 2013 -0700 + Merge pull request #512 from patelh/fix-kryo-serializer + [Fix reference bug in Kryo serializer, add test, update version] + + 557cfd0 Sun Mar 10 15:44:57 2013 -0700 + Merge pull request #515 from woggling/deploy-app-death + [Notify standalone deploy client of application death.] + + 04fb81f Sun Mar 3 17:20:07 2013 -0800 + Merge pull request #506 from rxin/spark-706 + [Fixed SPARK-706: Failures in block manager put leads to read task hanging.] + + 6cf4be4 Sun Mar 3 17:16:22 2013 -0800 + Merge pull request #462 from squito/stageInfo + [Track assorted metrics for each task, report summaries to user at stage completion] + + 6bfc7ca Sat Mar 2 22:14:49 2013 -0800 + Merge pull request #504 from mosharaf/master + [Worker address was getting removed when removing an app.] + + 94b3db1 Sat Mar 2 22:13:52 2013 -0800 + Merge pull request #508 from markhamstra/TestServerInUse + [Avoid bind failure in InputStreamsSuite] + + 25c71d3 Fri Mar 1 08:00:18 2013 -0800 + Merge pull request #507 from markhamstra/poms271 + [bump version to 0.7.1-SNAPSHOT in the subproject poms] + From 0f059bd62d1c840713fac0d9c6ee6d9165682c72 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 12:31:25 -0800 Subject: [PATCH 45/90] [maven-release-plugin] prepare release v0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7e9b201d3546d..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 44acbd9caf20a..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1264bc4933cfe..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 2ef2785e17fd3..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a19d8b3b450c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 83f9b23e12de2..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index d0e4428908056..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 86d2a0c050436..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 320d5326f3819..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d9168e33f6084..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From d0b9fce12d586c464306072fb210cb44b97dddd3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 12:31:33 -0800 Subject: [PATCH 46/90] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..361a1d8c1621c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..2be372da59d5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..90ac301261581 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..d12e3abf2e89b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..bd6a1c35f1a8c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..53f576c54e81d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..6f14fe87ae650 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..7a59d89819dd1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..910401691f0a8 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..66e492e76f873 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..002c731851e52 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml From 295734f6412be3d861fe0cc10b319063cc8909ab Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 12:40:18 -0800 Subject: [PATCH 47/90] Fix race condition in JobLoggerSuite --- .../scala/org/apache/spark/scheduler/JobLoggerSuite.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index 984881861c9a9..22babe8d2ef78 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -31,6 +31,8 @@ import org.apache.spark.rdd.RDD class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { + /** Length of time to wait while draining listener events. */ + val WAIT_TIMEOUT_MILLIS = 10000 test("inner method") { sc = new SparkContext("local", "joblogger") @@ -91,6 +93,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers sc.addSparkListener(joblogger) val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } rdd.reduceByKey(_+_).collect() + assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) @@ -119,8 +122,9 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers } sc.addSparkListener(joblogger) val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } - rdd.reduceByKey(_+_).collect() - + rdd.reduceByKey(_+_).collect() + assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + joblogger.onJobStartCount should be (1) joblogger.onJobEndCount should be (1) joblogger.onTaskEndCount should be (8) From a669605092451cdc93d3d872c6883f2673f2f5ea Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 12:53:58 -0800 Subject: [PATCH 48/90] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit d0b9fce12d586c464306072fb210cb44b97dddd3. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 361a1d8c1621c..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 2be372da59d5f..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 90ac301261581..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d12e3abf2e89b..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bd6a1c35f1a8c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 53f576c54e81d..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6f14fe87ae650..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 7a59d89819dd1..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 910401691f0a8..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 66e492e76f873..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 002c731851e52..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 1d3fa31ba6c07c7dde0c714b93beca8bc4a03e68 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 12:54:05 -0800 Subject: [PATCH 49/90] Revert "[maven-release-plugin] prepare release v0.8.1-incubating" This reverts commit 0f059bd62d1c840713fac0d9c6ee6d9165682c72. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..7e9b201d3546d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..44acbd9caf20a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..1264bc4933cfe 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..2ef2785e17fd3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..2a19d8b3b450c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..83f9b23e12de2 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..d0e4428908056 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..86d2a0c050436 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..320d5326f3819 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..21a6ba981262f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..d9168e33f6084 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml From fba873857133fb87cd53dc4cb0501eea1bd7edbf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 13:05:05 -0800 Subject: [PATCH 50/90] [maven-release-plugin] prepare release v0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7e9b201d3546d..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 44acbd9caf20a..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1264bc4933cfe..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 2ef2785e17fd3..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a19d8b3b450c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 83f9b23e12de2..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index d0e4428908056..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 86d2a0c050436..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 320d5326f3819..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d9168e33f6084..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 91505f3f2ace8b26e4dae90d362563bf2faa8fbf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 13:05:11 -0800 Subject: [PATCH 51/90] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..361a1d8c1621c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..2be372da59d5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..90ac301261581 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..d12e3abf2e89b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..bd6a1c35f1a8c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..53f576c54e81d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..6f14fe87ae650 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..7a59d89819dd1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..910401691f0a8 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..66e492e76f873 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..002c731851e52 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml From c76191457d82ad082e1a7c4e669e165d7684b2db Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 21:41:24 -0800 Subject: [PATCH 52/90] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 91505f3f2ace8b26e4dae90d362563bf2faa8fbf. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 361a1d8c1621c..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 2be372da59d5f..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 90ac301261581..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d12e3abf2e89b..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bd6a1c35f1a8c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 53f576c54e81d..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6f14fe87ae650..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 7a59d89819dd1..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 910401691f0a8..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 66e492e76f873..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 002c731851e52..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 473cba2ea37c50e1e1ad04f69379ce93c0ccb8d6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 21:41:28 -0800 Subject: [PATCH 53/90] Revert "[maven-release-plugin] prepare release v0.8.1-incubating" This reverts commit fba873857133fb87cd53dc4cb0501eea1bd7edbf. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..7e9b201d3546d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..44acbd9caf20a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..1264bc4933cfe 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..2ef2785e17fd3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..2a19d8b3b450c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..83f9b23e12de2 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..d0e4428908056 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..86d2a0c050436 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..320d5326f3819 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..21a6ba981262f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..d9168e33f6084 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml From c14f373d19f855e72461392d549899933131a85a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 22:34:34 -0800 Subject: [PATCH 54/90] Merge pull request #241 from pwendell/master Update broken links and add HDP 2.0 version string I ran a link checker on the UI and found several broken links. (cherry picked from commit 1f4a4bccf3cf7376c634bad2ebadfdd4c6f78195) Signed-off-by: Patrick Wendell --- docs/bagel-programming-guide.md | 2 +- docs/hadoop-third-party-distributions.md | 3 ++- docs/index.md | 2 +- docs/job-scheduling.md | 2 +- docs/running-on-yarn.md | 4 ++-- docs/streaming-programming-guide.md | 8 ++++---- 6 files changed, 11 insertions(+), 10 deletions(-) diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index 140190a38cfca..de001e6c52f85 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -106,7 +106,7 @@ _Example_ ## Operations -Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/incubator-spark/blob/master/bagel/src/main/scala/spark/bagel/Bagel.scala) for details. +Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/incubator-spark/blob/master/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala) for details. ### Actions diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index b33af2cf24d77..de6a2b0a43bd5 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -10,7 +10,7 @@ with these distributions: # Compile-time Hadoop Version When compiling Spark, you'll need to -[set the SPARK_HADOOP_VERSION flag](http://localhost:4000/index.html#a-note-about-hadoop-versions): +[set the SPARK_HADOOP_VERSION flag](index.html#a-note-about-hadoop-versions): SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly @@ -40,6 +40,7 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors. HDP 1.21.1.2 HDP 1.11.0.3 HDP 1.01.0.3 + HDP 2.02.2.0 diff --git a/docs/index.md b/docs/index.md index 45616f78f99fd..d3ac696d1e818 100644 --- a/docs/index.md +++ b/docs/index.md @@ -58,7 +58,7 @@ By default, Spark links to Hadoop 1.0.4. You can change this by setting the SPARK_HADOOP_VERSION=2.2.0 sbt/sbt assembly -In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set +In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set `SPARK_YARN` to `true`: SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index d304c5497bdb3..dbcb9ae343357 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -91,7 +91,7 @@ The fair scheduler also supports grouping jobs into _pools_, and setting differe (e.g. weight) for each pool. This can be useful to create a "high-priority" pool for more important jobs, for example, or to group the jobs of each user together and give _users_ equal shares regardless of how many concurrent jobs they have instead of giving _jobs_ equal shares. This approach is modeled after the -[Hadoop Fair Scheduler](http://hadoop.apache.org/docs/stable/fair_scheduler.html). +[Hadoop Fair Scheduler](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/FairScheduler.html). Without any intervention, newly submitted jobs go into a _default pool_, but jobs' pools can be set by adding the `spark.scheduler.pool` "local property" to the SparkContext in the thread that's submitting them. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index ae65127759c28..9e4c4e123f0af 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -116,7 +116,7 @@ For example: Hadoop 2.2.x users must build Spark and publish it locally. The SBT build process handles Hadoop 2.2.x as a special case. This version of Hadoop has new YARN API changes and depends on a Protobuf version (2.5) that is not compatible with the Akka version (2.0.5) that Spark uses. Therefore, if the Hadoop version (e.g. set through ```SPARK_HADOOP_VERSION```) starts with 2.2.0 or higher then the build process will depend on Akka artifacts distributed by the Spark project compatible with Protobuf 2.5. Furthermore, the build process then uses the directory ```new-yarn``` (instead of ```yarn```), which supports the new YARN API. The build process should seamlessly work out of the box. -See [Building Spark with Maven](building-with-maven.md) for instructions on how to build Spark using the Maven process. +See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process. # Important Notes @@ -124,4 +124,4 @@ See [Building Spark with Maven](building-with-maven.md) for instructions on how - The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored. - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN. - The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. -- YARN 2.2.x users cannot simply depend on the Spark packages without building Spark, as the published Spark artifacts are compiled to work with the pre 2.2 API. Those users must build Spark and publish it locally. \ No newline at end of file +- YARN 2.2.x users cannot simply depend on the Spark packages without building Spark, as the published Spark artifacts are compiled to work with the pre 2.2 API. Those users must build Spark and publish it locally. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 851e30fe761af..82f42e0b8d16a 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -214,7 +214,7 @@ ssc.stop() {% endhighlight %} # Example -A simple example to start off is the [NetworkWordCount](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `/streaming/src/main/scala/spark/streaming/examples/NetworkWordCount.scala` . +A simple example to start off is the [NetworkWordCount](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `/streaming/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala` . {% highlight scala %} import org.apache.spark.streaming.{Seconds, StreamingContext} @@ -283,7 +283,7 @@ Time: 1357008430000 ms -You can find more examples in `/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. +You can find more examples in `/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. # DStream Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`. @@ -483,7 +483,7 @@ Similar to [Spark's Java API](java-programming-guide.html), we also provide a Ja 1. Functions for transformations must be implemented as subclasses of [Function](api/core/index.html#org.apache.spark.api.java.function.Function) and [Function2](api/core/index.html#org.apache.spark.api.java.function.Function2) 1. Unlike the Scala API, the Java API handles DStreams for key-value pairs using a separate [JavaPairDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaPairDStream) class(similar to [JavaRDD and JavaPairRDD](java-programming-guide.html#rdd-classes). DStream functions like `map` and `filter` are implemented separately by JavaDStreams and JavaPairDStream to return DStreams of appropriate types. -Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java` +Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java` The streaming context and the socket stream from input source is started by using a `JavaStreamingContext`, that has the same parameters and provides the same input streams as its Scala counterpart. @@ -527,5 +527,5 @@ JavaPairDStream wordCounts = words.map( # Where to Go from Here * API docs - [Scala](api/streaming/index.html#org.apache.spark.streaming.package) and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package) -* More examples - [Scala](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/spark/streaming/examples) +* More examples - [Scala](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) and [Java](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/streaming/examples) * [Paper describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) From 871ab60a40606fb2a5e644201542b0d11db770e8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 7 Dec 2013 22:49:47 -0800 Subject: [PATCH 55/90] Use consistent CHANGES.txt format --- CHANGES.txt | 167 ++++++++++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 155 insertions(+), 12 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 60b1e8ba78ff7..0ab59f363af20 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -2,410 +2,553 @@ Spark Change Log Release 0.8.1-incubating + c14f373 Sat Dec 7 22:35:31 2013 -0800 + Merge pull request #241 from pwendell/master + [Update broken links and add HDP 2.0 version string] + + 9c9e71e Sat Dec 7 12:47:26 2013 -0800 + Merge pull request #241 from pwendell/branch-0.8 + [Fix race condition in JobLoggerSuite [0.8 branch]] + 92597c0 Sat Dec 7 11:58:00 2013 -0800 Merge pull request #240 from pwendell/master - + [SPARK-917 Improve API links in nav bar] + cfca70e Sat Dec 7 01:15:20 2013 -0800 Merge pull request #236 from pwendell/shuffle-docs - + [Adding disclaimer for shuffle file consolidation] + 80cc4ff Sat Dec 7 01:15:20 2013 -0800 Merge pull request #237 from pwendell/formatting-fix - + [Formatting fix] + 4a6aae3 Sat Dec 7 01:15:20 2013 -0800 Merge pull request #235 from pwendell/master - + [Minor doc fixes and updating README] + 2642312 Sat Dec 7 01:15:20 2013 -0800 Merge pull request #234 from alig/master - + [Updated documentation about the YARN v2.2 build process] + 2d3eae2 Sat Dec 7 01:15:19 2013 -0800 Merge pull request #199 from harveyfeng/yarn-2.2 - + [Hadoop 2.2 migration] + 1e9d084 Sat Dec 7 01:15:19 2013 -0800 Merge pull request #101 from colorant/yarn-client-scheduler - + [For SPARK-527, Support spark-shell when running on YARN] + 20d1f8b Sat Dec 7 01:15:09 2013 -0800 Merge pull request #191 from hsaputra/removesemicolonscala - + [Cleanup to remove semicolons (;) from Scala code] + 2b76315 Sat Dec 7 00:02:14 2013 -0800 Merge pull request #178 from hsaputra/simplecleanupcode - + [Simple cleanup on Spark's Scala code] + ee22be0 Fri Dec 6 23:29:38 2013 -0800 Merge pull request #189 from tgravescs/sparkYarnErrorHandling - + [Impove Spark on Yarn Error handling] + d77c337 Thu Dec 5 23:30:11 2013 -0800 Merge pull request #232 from markhamstra/FiniteWait - + [jobWaiter.synchronized before jobWaiter.wait] + 17ca8a1 Thu Dec 5 14:32:01 2013 -0800 Merge pull request #231 from pwendell/branch-0.8 - + [Bumping version numbers for 0.8.1 release] + 47fce43 Thu Dec 5 12:33:02 2013 -0800 Merge pull request #228 from pwendell/master + [Document missing configs and set shuffle consolidation to false.] cc33f9f Wed Dec 4 15:57:47 2013 -0800 Merge pull request #227 from pwendell/master + [Fix small bug in web UI and minor clean-up.] ba44f21 Wed Dec 4 15:56:58 2013 -0800 Merge pull request #223 from rxin/transient + [Mark partitioner, name, and generator field in RDD as @transient.] 31da065 Wed Dec 4 14:01:13 2013 -0800 Merge pull request #95 from aarondav/perftest + [Minor: Put StoragePerfTester in org/apache/] daaaee1 Tue Dec 3 14:22:05 2013 -0800 Merge pull request #218 from JoshRosen/spark-970-pyspark-unicode-error + [Fix UnicodeEncodeError in PySpark saveAsTextFile() (SPARK-970)] 8b091fe Mon Dec 2 21:28:13 2013 -0800 Merge pull request #181 from BlackNiuza/fix_tasks_number + [correct number of tasks in ExecutorsUI] d21266e Sun Dec 1 12:47:30 2013 -0800 Merge pull request #219 from sundeepn/schedulerexception + [Scheduler quits when newStage fails] be9c176 Tue Nov 26 10:27:41 2013 -0800 Merge pull request #201 from rxin/mappartitions + [Use the proper partition index in mapPartitionsWIthIndex] 9949561 Mon Nov 25 07:51:23 2013 +0800 Merge pull request #197 from aarondav/patrick-fix + [Fix 'timeWriting' stat for shuffle files] c59ce18 Sun Nov 24 11:04:00 2013 +0800 Merge pull request #200 from mateiz/hash-fix + [AppendOnlyMap fixes] d7ab87e Fri Nov 22 10:27:16 2013 +0800 Merge pull request #193 from aoiwelle/patch-1 + [Fix Kryo Serializer buffer documentation inconsistency] d7c6a00 Fri Nov 22 10:13:37 2013 +0800 Merge pull request #196 from pwendell/master + [TimeTrackingOutputStream should pass on calls to close() and flush().] 37126e8 Sun Nov 17 18:53:16 2013 -0800 Merge pull request #174 from ahirreddy/master + [Write Spark UI url to driver file on HDFS] 8823057 Sun Nov 17 18:53:04 2013 -0800 Merge pull request #166 from ahirreddy/simr-spark-ui + [SIMR Backend Scheduler will now write Spark UI URL to HDFS, which is to ...] e134ed5 Sun Nov 17 18:51:19 2013 -0800 Merge pull request #137 from tgravescs/sparkYarnJarsHdfsRebase + [Allow spark on yarn to be run from HDFS.] af98fbc Sun Nov 17 18:50:23 2013 -0800 Merge pull request #165 from NathanHowell/kerberos-master + [spark-assembly.jar fails to authenticate with YARN ResourceManager] a64397b Sun Nov 17 18:49:40 2013 -0800 Merge pull request #153 from ankurdave/stop-spot-cluster + [Enable stopping and starting a spot cluster] 6c60768 Sun Nov 17 18:46:40 2013 -0800 Merge pull request #160 from xiajunluan/JIRA-923 + [Fix bug JIRA-923] f0d350a Sun Nov 17 18:44:49 2013 -0800 Merge pull request #175 from kayousterhout/no_retry_not_serializable + [Don't retry tasks when they fail due to a NotSerializableException] 9d56371 Sun Nov 17 18:43:59 2013 -0800 Merge pull request #173 from kayousterhout/scheduler_hang + [Fix bug where scheduler could hang after task failure.] 24e238b Sun Nov 17 18:42:41 2013 -0800 Merge pull request #182 from rxin/vector + [Slightly enhanced PrimitiveVector:] e7927ad Thu Nov 14 10:32:27 2013 -0800 Merge pull request #169 from kayousterhout/mesos_fix + [Don't ignore spark.cores.max when using Mesos Coarse mode] 41dc566 Thu Nov 14 10:30:57 2013 -0800 Merge pull request #170 from liancheng/hadooprdd-doc-typo + [Fixed a scaladoc typo in HadoopRDD.scala] 333859f Thu Nov 14 10:26:27 2013 -0800 Merge pull request #171 from RIA-pierre-borckmans/master + [Fixed typos in the CDH4 distributions version codes.] c856651 Tue Nov 12 10:22:18 2013 -0800 Merge pull request #164 from tdas/kafka-fix + [Made block generator thread safe to fix Kafka bug.] 30786c6 Sun Nov 10 11:58:58 2013 -0800 Merge pull request #157 from rxin/kryo + [3 Kryo related changes.] 5ce6c75 Sat Nov 9 22:31:59 2013 -0800 Merge pull request #147 from JoshRosen/fix-java-api-completeness-checker + [Add spark-tools assembly to spark-class'ss classpath] 1d52b50 Sat Nov 9 15:47:40 2013 -0800 Merge pull request #149 from tgravescs/fixSecureHdfsAccess + [Fix secure hdfs access for spark on yarn] 32a0c4f Sat Nov 9 15:40:54 2013 -0800 Merge pull request #155 from rxin/jobgroup + [Don't reset job group when a new job description is set.] 07ae524 Sat Nov 9 11:56:46 2013 -0800 Merge pull request #152 from rxin/repl + [Propagate SparkContext local properties from spark-repl caller thread to the repl execution thread.] a5916b9 Thu Nov 7 11:08:44 2013 -0800 Merge pull request #148 from squito/include_appId + [Include appId in executor cmd line args] d5ae953 Wed Nov 6 23:23:12 2013 -0800 Merge pull request #23 from jerryshao/multi-user + [Add Spark multi-user support for standalone mode and Mesos] 1d9412b Wed Nov 6 13:28:02 2013 -0800 Merge pull request #144 from liancheng/runjob-clean + [Removed unused return value in SparkContext.runJob] c8e0c0d Wed Nov 6 09:40:00 2013 -0800 Merge pull request #145 from aarondav/sls-fix + [Attempt to fix SparkListenerSuite breakage] 0f62786 Tue Nov 5 23:14:28 2013 -0800 Merge pull request #143 from rxin/scheduler-hang + [Ignore a task update status if the executor doesn't exist anymore.] 96670e7 Tue Nov 5 10:43:10 2013 -0800 Merge pull request #140 from aarondav/merge-75 + [Manually merge BlockManager refactor #75] 0848167 Tue Nov 5 10:42:36 2013 -0800 Merge pull request #142 from liancheng/dagscheduler-pattern-matching + [Using case class deep match to simplify code in DAGScheduler.processEvent] 07b3f01 Mon Nov 4 23:32:56 2013 -0800 Merge pull request #75 from JoshRosen/block-manager-cleanup + [Code de-duplication in BlockManager] e80d1cf Mon Nov 4 20:47:44 2013 -0800 Merge pull request #139 from aarondav/shuffle-next + [Never store shuffle blocks in BlockManager] 518cf22 Mon Nov 4 18:21:27 2013 -0800 Merge pull request #128 from shimingfei/joblogger-doc + [add javadoc to JobLogger, and some small fix] 7e00dee Mon Nov 4 17:54:35 2013 -0800 Merge pull request #130 from aarondav/shuffle + [Memory-optimized shuffle file consolidation] 1d11e43 Mon Nov 4 16:30:30 2013 -0800 Merge pull request #138 from marmbrus/branch-0.8 + [Fix build break caused by BlockInfo refactor] a3544ee Sun Nov 3 23:49:19 2013 -0800 Merge pull request #70 from rxin/hash1 + [Fast, memory-efficient hash set, hash table implementations optimized for primitive data types.] e094daf Sun Nov 3 23:48:40 2013 -0800 Merge pull request #129 from velvia/2013-11/document-local-uris + [Document & finish support for local: URIs] a9e7787 Sun Nov 3 23:48:26 2013 -0800 Merge pull request #125 from velvia/2013-10/local-jar-uri + [Add support for local:// URI scheme for addJars()] 57fdb3f Sun Nov 3 23:46:18 2013 -0800 Merge pull request #117 from stephenh/avoid_concurrent_modification_exception + [Handle ConcurrentModificationExceptions in SparkContext init.] ec0e4f0 Sun Nov 3 23:45:23 2013 -0800 Merge pull request #124 from tgravescs/sparkHadoopUtilFix + [Pull SparkHadoopUtil out of SparkEnv (jira SPARK-886)] ba0e858 Sun Nov 3 23:43:48 2013 -0800 Merge pull request #126 from kayousterhout/local_fix + [Fixed incorrect log message in local scheduler] 504fe74 Sun Nov 3 23:43:03 2013 -0800 Merge pull request #118 from JoshRosen/blockinfo-memory-usage + [Reduce the memory footprint of BlockInfo objects] 25fa229 Sun Nov 3 23:41:19 2013 -0800 Merge pull request #112 from kayousterhout/ui_task_attempt_id + [Display both task ID and task attempt ID in UI, and rename taskId to taskAttemptId] 57ea854 Sun Nov 3 23:38:37 2013 -0800 Merge pull request #102 from tdas/transform + [Added new Spark Streaming operations] 5b45c9b Sat Nov 2 14:42:22 2013 -0700 Merge pull request #133 from Mistobaan/link_fix + [update default github] 054d97b Sat Nov 2 14:37:35 2013 -0700 Merge pull request #134 from rxin/readme + [Fixed a typo in Hadoop version in README.] 87d4e1c Fri Nov 1 17:58:38 2013 -0700 Merge pull request #132 from Mistobaan/doc_fix + [fix persistent-hdfs] 3db505c Tue Oct 29 01:42:07 2013 -0400 Merge pull request #119 from soulmachine/master + [A little revise for the document] abeca01 Sun Oct 27 22:11:39 2013 -0400 Merge pull request #115 from aarondav/shuffle-fix + [Eliminate extra memory usage when shuffle file consolidation is disabled] 79e5c50 Sat Oct 26 13:05:40 2013 -0700 Merge pull request #108 from alig/master + [Changes to enable executing by using HDFS as a synchronization point between driver and executors, as well as ensuring executors exit properly.] cb24278 Sat Oct 26 11:41:18 2013 -0700 Merge pull request #113 from pwendell/master + [Improve error message when multiple assembly jars are present.] 3ec2f51 Sat Oct 26 11:39:29 2013 -0700 Merge pull request #111 from kayousterhout/ui_name + [Properly display the name of a stage in the UI.] 7d47704 Sat Oct 26 11:36:23 2013 -0700 Merge pull request #114 from soulmachine/master + [A little revise for the document] 148509f Thu Oct 24 22:32:23 2013 -0700 Merge pull request #109 from pwendell/master + [Adding Java/Java Streaming versions of `repartition` with associated tests] c018c61 Thu Oct 24 17:27:28 2013 -0700 Merge pull request #97 from ewencp/pyspark-system-properties + [Add classmethod to SparkContext to set system properties.] 3500b66 Thu Oct 24 17:27:11 2013 -0700 Merge pull request #93 from kayousterhout/ui_new_state + [Show "GETTING_RESULTS" state in UI.] f5eff85 Thu Oct 24 17:26:44 2013 -0700 Merge pull request #83 from ewencp/pyspark-accumulator-add-method + [Add an add() method to pyspark accumulators.] 59d6f06 Thu Oct 24 17:09:05 2013 -0700 Merge pull request #106 from pwendell/master + [Add a `repartition` operator.] 76f3c2f Wed Oct 23 18:03:26 2013 -0700 Merge pull request #103 from JoshRosen/unpersist-fix + [Add unpersist() to JavaDoubleRDD and JavaPairRDD.] 534bab2 Tue Oct 22 16:02:45 2013 -0700 Merge pull request #100 from JoshRosen/spark-902 + [Remove redundant Java Function call() definitions] fa9a0e4 Tue Oct 22 12:23:17 2013 -0700 Merge pull request #90 from pwendell/master + [SPARK-940: Do not directly pass Stage objects to SparkListener.] c449ee1 Tue Oct 22 00:00:35 2013 -0700 Merge pull request #92 from tgravescs/sparkYarnFixClasspath + [Fix the Worker to use CoarseGrainedExecutorBackend and modify classpath ...] 498cc6b Mon Oct 21 22:45:31 2013 -0700 Merge pull request #87 from aarondav/shuffle-base + [Basic shuffle file consolidation] e3ad6a5 Mon Oct 21 20:20:42 2013 -0700 Revert "Merge pull request #94 from aarondav/mesos-fix" + [This reverts commit fe974ba1484dbe6fffdae8680b3b37b26de83145.] fe974ba Mon Oct 21 20:15:30 2013 -0700 Merge pull request #94 from aarondav/mesos-fix + [Fix mesos urls] 1c3f4bd Mon Oct 21 11:58:22 2013 -0700 Merge pull request #88 from rxin/clean + [Made the following traits/interfaces/classes non-public:] 1a50c79 Sun Oct 20 22:27:29 2013 -0700 Merge pull request #41 from pwendell/shuffle-benchmark + [Provide Instrumentation for Shuffle Write Performance] 37a755c Sun Oct 20 21:04:33 2013 -0700 Merge pull request #89 from rxin/executor + [Don't setup the uncaught exception handler in local mode.] ec74428 Sun Oct 20 11:46:14 2013 -0700 Merge pull request #84 from rxin/kill1 + [Added documentation for setJobGroup. Also some minor cleanup in SparkContext.] 52d13a6 Sun Oct 20 10:50:54 2013 -0700 Merge pull request #85 from rxin/clean + [Moved the top level spark package object from spark to org.apache.spark] 919c557 Sat Oct 19 11:29:00 2013 -0700 Merge pull request #79 from aarondav/scdefaults0.8 + [Add deprecation warning to unrecognized MASTER urls] f3de2ce Fri Oct 18 23:20:16 2013 -0700 Merge pull request #76 from pwendell/master + [Clarify compression property.] f181560 Fri Oct 18 23:14:28 2013 -0700 Merge pull request #68 from mosharaf/master + [Faster and stable/reliable broadcast] eaa2150 Fri Oct 18 23:08:47 2013 -0700 Merge pull request #74 from rxin/kill + [Job cancellation via job group id.] df21ac8 Thu Oct 17 18:38:46 2013 -0700 Merge pull request #69 from KarthikTunga/master + [Fix for issue SPARK-627. Implementing --config argument in the scripts.] b531552 Thu Oct 17 18:37:22 2013 -0700 Merge pull request #67 from kayousterhout/remove_tsl + [Removed TaskSchedulerListener interface.] 5a73ab7 Thu Oct 17 18:36:36 2013 -0700 Merge pull request #62 from harveyfeng/master + [Make TaskContext's stageId publicly accessible.] b6ce111 Thu Oct 17 18:35:33 2013 -0700 Merge pull request #61 from kayousterhout/daemon_thread + [Unified daemon thread pools] 2760055 Thu Oct 17 18:34:56 2013 -0700 Merge pull request #59 from rxin/warning + [Bump up logging level to warning for failed tasks.] 1e67234 Thu Oct 17 18:33:21 2013 -0700 Merge pull request #65 from tgravescs/fixYarn + [Fix yarn build] d0c9d41 Thu Oct 17 18:32:54 2013 -0700 Merge pull request #34 from kayousterhout/rename + [Renamed StandaloneX to CoarseGrainedX.] cee3b43 Wed Oct 16 10:37:55 2013 -0700 Merge pull request #63 from pwendell/master + [Fixing spark streaming example and a bug in examples build.] 1d92983 Tue Oct 15 10:52:15 2013 -0700 Merge pull request #58 from hsaputra/update-pom-asf + [Update pom.xml to use version 13 of the ASF parent pom] c50b016 Mon Oct 14 23:18:37 2013 -0700 Merge pull request #29 from rxin/kill + [Job killing] 616ea6f Mon Oct 14 20:59:31 2013 -0700 Merge pull request #57 from aarondav/bid + [Refactor BlockId into an actual type] e306de8 Mon Oct 14 20:58:48 2013 -0700 Merge pull request #53 from witgo/master + [Add a zookeeper compile dependency to fix build in maven] d66c01f Mon Oct 14 20:58:17 2013 -0700 Merge pull request #19 from aarondav/master-zk + [Standalone Scheduler fault tolerance using ZooKeeper] 0fcb234 Mon Oct 14 20:54:34 2013 -0700 Merge pull request #46 from mateiz/py-sort-update + [Fix PySpark docs and an overly long line of code after #38] f94aa52 Mon Oct 14 20:53:15 2013 -0700 Merge pull request #44 from mateiz/fast-map + [A fast and low-memory append-only map for shuffle operations] 36f5b08 Mon Oct 14 20:50:28 2013 -0700 Merge pull request #33 from AndreSchumacher/pyspark_partition_key_change + [Fixing SPARK-602: PythonPartitioner] d615b14 Mon Oct 14 20:48:45 2013 -0700 Merge pull request #32 from mridulm/master + [Address review comments, move to incubator spark] 7084217 Mon Oct 14 19:31:22 2013 -0700 Merge pull request #27 from davidmccauley/master + [SPARK-920/921 - JSON endpoint updates] a35259a Mon Oct 14 19:28:46 2013 -0700 Merge pull request #26 from Du-Li/master + [fixed a wildcard bug in make-distribution.sh; ask sbt to check local] 62ce4ae Mon Oct 14 19:28:09 2013 -0700 Merge pull request #25 from CruncherBigData/master + [Update README: updated the link] 6961744 Mon Oct 14 18:55:41 2013 -0700 Merge pull request #10 from kayousterhout/results_through-bm + [Send Task results through the block manager when larger than Akka frame size (fixes SPARK-669).] ce364c0 Mon Oct 14 18:54:35 2013 -0700 Merge pull request #4 from MLnick/implicit-als + [Adding algorithm for implicit feedback data to ALS] f94bd3f Mon Oct 14 15:10:59 2013 -0700 Merge pull request #28 from tgravescs/sparYarnAppName + [Allow users to set the application name for Spark on Yarn] dc2c90d Mon Oct 14 15:03:34 2013 -0700 Merge pull request #38 from AndreSchumacher/pyspark_sorting + [SPARK-705: implement sortByKey() in PySpark] 00a7551 Sat Oct 12 21:26:22 2013 -0700 Merge pull request #52 from harveyfeng/hadoop-closure + [Add an optional closure parameter to HadoopRDD instantiation to use when creating local JobConfs.] 5383a5a Sat Oct 12 21:26:11 2013 -0700 Merge pull request #20 from harveyfeng/hadoop-config-cache + [Allow users to pass broadcasted Configurations and cache InputFormats across Hadoop file reads.] 4a2e76a Fri Oct 11 16:09:07 2013 -0700 Merge pull request #54 from aoiwelle/remove_unused_imports + [Remove unnecessary mutable imports] 64fae16 Wed Oct 9 21:42:18 2013 -0700 Merge pull request #47 from xiliu82/branch-0.8 + [add Function3 and WrappedFunction3] dfc62e2 Wed Oct 9 16:55:58 2013 -0700 Merge pull request #49 from mateiz/kryo-fix-2 + [Fix Chill serialization of Range objects] 0b6f047 Wed Oct 9 16:53:31 2013 -0700 Merge pull request #50 from kayousterhout/SPARK-908 + [Fix race condition in SparkListenerSuite (fixes SPARK-908).] f930dd4 Tue Oct 8 22:58:35 2013 -0700 Merge pull request #43 from mateiz/kryo-fix + [Don't allocate Kryo buffers unless needed] 8e9bd93 Mon Oct 7 20:47:09 2013 -0700 Merge pull request #42 from pwendell/shuffle-read-perf + [Fix inconsistent and incorrect log messages in shuffle read path] f2cdcc4 Mon Oct 7 15:49:32 2013 -0700 Merge pull request #40 from pwendell/branch-0.8 + [Adding Shark 0.7.1 to EC2 scripts [branch 0.8 version]] 023e3fd Mon Oct 7 10:47:45 2013 -0700 Merge pull request #31 from sundeepn/branch-0.8 + [Resolving package conflicts with hadoop 0.23.9] 3cb9040 Thu Sep 26 14:37:06 2013 -0700 Merge pull request #17 from rxin/optimize + [Remove -optimize flag] 35bcf32 Thu Sep 26 14:17:00 2013 -0700 Merge pull request #930 from holdenk/master + [Add mapPartitionsWithIndex] 976fe60 Thu Sep 26 14:16:17 2013 -0700 Merge pull request #14 from kayousterhout/untangle_scheduler + [Improved organization of scheduling packages.] 8cbc96b Thu Sep 26 13:16:05 2013 -0700 Merge pull request #7 from wannabeast/memorystore-fixes + [some minor fixes to MemoryStore] 240ca93 Thu Sep 26 13:12:06 2013 -0700 Merge pull request #9 from rxin/limit + [Smarter take/limit implementation.] a186792 Thu Sep 26 13:10:59 2013 -0700 Merge pull request #937 from jerryshao/localProperties-fix + [Fix PR926 local properties issues in Spark Streaming like scenarios] f3c60c9 Thu Sep 26 13:10:24 2013 -0700 Merge pull request #941 from ilikerps/master + [Add "org.apache." prefix to packages in spark-class] d5a8dbf Thu Sep 26 13:09:30 2013 -0700 Merge pull request #928 from jerryshao/fairscheduler-refactor + [Refactor FairSchedulableBuilder] Release 0.8.0-incubating From e88e6369d9ac55dff75c230ed5bc96c995b1d620 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 02:35:29 -0800 Subject: [PATCH 56/90] [maven-release-plugin] prepare release v0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7e9b201d3546d..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 44acbd9caf20a..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1264bc4933cfe..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 2ef2785e17fd3..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a19d8b3b450c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 83f9b23e12de2..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index d0e4428908056..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 86d2a0c050436..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 320d5326f3819..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d9168e33f6084..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 7f7ac64e2847b7cbdb9274fae75fed981601e7d7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 02:35:36 -0800 Subject: [PATCH 57/90] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..361a1d8c1621c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..2be372da59d5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..90ac301261581 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..d12e3abf2e89b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..bd6a1c35f1a8c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..53f576c54e81d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..6f14fe87ae650 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..7a59d89819dd1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..910401691f0a8 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..66e492e76f873 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..002c731851e52 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml From 408f50bbbe0674126582edbc953313eb8c89d438 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 11:11:45 -0800 Subject: [PATCH 58/90] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 7f7ac64e2847b7cbdb9274fae75fed981601e7d7. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 361a1d8c1621c..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 2be372da59d5f..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 90ac301261581..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d12e3abf2e89b..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bd6a1c35f1a8c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 53f576c54e81d..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6f14fe87ae650..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 7a59d89819dd1..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 910401691f0a8..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 66e492e76f873..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 002c731851e52..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From c7058d1086f64def8553dc836d990e1b85389d1b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 11:11:49 -0800 Subject: [PATCH 59/90] Revert "[maven-release-plugin] prepare release v0.8.1-incubating" This reverts commit e88e6369d9ac55dff75c230ed5bc96c995b1d620. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..7e9b201d3546d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..44acbd9caf20a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..1264bc4933cfe 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..2ef2785e17fd3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..2a19d8b3b450c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..83f9b23e12de2 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..d0e4428908056 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..86d2a0c050436 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..320d5326f3819 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..21a6ba981262f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..d9168e33f6084 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml From 1bc7259f61e93aa7d9d155864e331bad9e5d2f0d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 11:12:03 -0800 Subject: [PATCH 60/90] Minor documentation fixes --- README.md | 8 +++----- docs/building-with-maven.md | 2 +- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 37d9c0ffe7b23..5e3a65f3401c7 100644 --- a/README.md +++ b/README.md @@ -55,7 +55,7 @@ versions without YARN, use: # Cloudera CDH 4.2.0 with MapReduce v1 $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly -For Apache Hadoop 2.0.X, 2.1.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions +For Apache Hadoop 2.0.X, 2.1.X, 2.2.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, also set `SPARK_YARN=true`: # Apache Hadoop 2.0.5-alpha @@ -64,10 +64,8 @@ with YARN, also set `SPARK_YARN=true`: # Cloudera CDH 4.2.0 with MapReduce v2 $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly -When building for Hadoop 2.2.X and newer, you'll need to include the additional `new-yarn` profile: - - # Apache Hadoop 2.2.X and newer - $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn + # Apache Hadoop 2.2.0 with YARN + $ SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt assembly For convenience, these variables may also be set through the `conf/spark-env.sh` file described below. diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index c709001632261..b2744b002cca3 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -48,7 +48,7 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with Hadoop versions 2.2.x and newer can be built by setting the ```new-yarn``` and the ```yarn.version``` as follows: # Apache Hadoop 2.2.X and newer - $ mvn -Dyarn.version=2.2.0 -Dhadoop.version=2.2.0 -Pnew-yarn + $ mvn -Pnew-yarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package The build process handles Hadoop 2.2.x as a special case that uses the directory ```new-yarn```, which supports the new YARN API. Furthermore, for this version, the build depends on artifacts published by the spark-project to enable Akka 2.0.5 to work with protobuf 2.5. From bf23794a766d4f94076d2417f128f15465f25495 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 11:37:06 -0800 Subject: [PATCH 61/90] [maven-release-plugin] prepare release v0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7e9b201d3546d..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 44acbd9caf20a..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1264bc4933cfe..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 2ef2785e17fd3..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a19d8b3b450c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 83f9b23e12de2..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index d0e4428908056..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 86d2a0c050436..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 320d5326f3819..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d9168e33f6084..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 4ece27df4243a7b0ba2791c3c7bece5aed86d342 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 11:37:12 -0800 Subject: [PATCH 62/90] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..361a1d8c1621c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..2be372da59d5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..90ac301261581 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..d12e3abf2e89b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..bd6a1c35f1a8c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..53f576c54e81d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..6f14fe87ae650 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..7a59d89819dd1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..910401691f0a8 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..66e492e76f873 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..002c731851e52 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml From dcc678f1efcc655279cb761e3850068cede98714 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 20:50:49 -0800 Subject: [PATCH 63/90] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 4ece27df4243a7b0ba2791c3c7bece5aed86d342. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 361a1d8c1621c..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 2be372da59d5f..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 90ac301261581..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d12e3abf2e89b..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bd6a1c35f1a8c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 53f576c54e81d..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6f14fe87ae650..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 7a59d89819dd1..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 910401691f0a8..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 66e492e76f873..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 002c731851e52..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 71d76a068cdb618dd38de76f8ba08c95cb631651 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 20:50:52 -0800 Subject: [PATCH 64/90] Revert "[maven-release-plugin] prepare release v0.8.1-incubating" This reverts commit bf23794a766d4f94076d2417f128f15465f25495. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..7e9b201d3546d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..44acbd9caf20a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..1264bc4933cfe 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..2ef2785e17fd3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..2a19d8b3b450c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..83f9b23e12de2 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..d0e4428908056 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..86d2a0c050436 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..320d5326f3819 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..21a6ba981262f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..d9168e33f6084 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml From 00d5c734dd4b12e128295518e4bd620fdb13bed7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 21:24:49 -0800 Subject: [PATCH 65/90] [maven-release-plugin] prepare release v0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7e9b201d3546d..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 44acbd9caf20a..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1264bc4933cfe..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 2ef2785e17fd3..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a19d8b3b450c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 83f9b23e12de2..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index d0e4428908056..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 86d2a0c050436..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 320d5326f3819..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d9168e33f6084..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 4c555328dd782efc2ab97ae35ea2f3a5b00cb450 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 21:25:06 -0800 Subject: [PATCH 66/90] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..361a1d8c1621c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..2be372da59d5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..90ac301261581 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..d12e3abf2e89b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..bd6a1c35f1a8c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..53f576c54e81d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..6f14fe87ae650 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..7a59d89819dd1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..910401691f0a8 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..66e492e76f873 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..002c731851e52 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml From 7a72b6084e3f011755ebaa29e78dd3ea52794105 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 21:36:27 -0800 Subject: [PATCH 67/90] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 4c555328dd782efc2ab97ae35ea2f3a5b00cb450. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 361a1d8c1621c..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 2be372da59d5f..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 90ac301261581..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d12e3abf2e89b..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bd6a1c35f1a8c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 53f576c54e81d..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6f14fe87ae650..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 7a59d89819dd1..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 910401691f0a8..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 66e492e76f873..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 002c731851e52..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From fde53474386a3679603e8020e825e6e61289511b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 21:36:30 -0800 Subject: [PATCH 68/90] Revert "[maven-release-plugin] prepare release v0.8.1-incubating" This reverts commit 00d5c734dd4b12e128295518e4bd620fdb13bed7. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..7e9b201d3546d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..44acbd9caf20a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..1264bc4933cfe 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..2ef2785e17fd3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..2a19d8b3b450c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..83f9b23e12de2 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..d0e4428908056 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..86d2a0c050436 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..320d5326f3819 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..21a6ba981262f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..d9168e33f6084 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml From 4b2769bd6b4f13793e0b853ccc000b153d55e7d1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 20:49:20 -0800 Subject: [PATCH 69/90] Merge pull request #195 from dhardy92/fix_DebScriptPackage [Deb] fix package of Spark classes adding org.apache prefix in scripts embeded in .deb (cherry picked from commit d992ec6d9be30e624c8edb2a50c193ac3cfbab7a) Signed-off-by: Patrick Wendell --- repl-bin/src/deb/bin/spark-executor | 2 +- repl-bin/src/deb/bin/spark-shell | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor index bcfae22677fdd..052d76fb8d81c 100755 --- a/repl-bin/src/deb/bin/spark-executor +++ b/repl-bin/src/deb/bin/spark-executor @@ -19,4 +19,4 @@ FWDIR="$(cd `dirname $0`; pwd)" echo "Running spark-executor with framework dir = $FWDIR" -exec $FWDIR/run spark.executor.MesosExecutorBackend +exec $FWDIR/run org.apache.spark.executor.MesosExecutorBackend diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell index ec7e33e1e3dec..118349d7c30f2 100755 --- a/repl-bin/src/deb/bin/spark-shell +++ b/repl-bin/src/deb/bin/spark-shell @@ -18,4 +18,4 @@ # FWDIR="$(cd `dirname $0`; pwd)" -exec $FWDIR/run spark.repl.Main "$@" +exec $FWDIR/run org.apache.spark.repl.Main "$@" From 5ab8e0408886b4176d53659c1fc8bbbcdeec13fc Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 21:34:53 -0800 Subject: [PATCH 70/90] Updating CHANGES file --- CHANGES.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGES.txt b/CHANGES.txt index 0ab59f363af20..ce0aeab524505 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -2,6 +2,10 @@ Spark Change Log Release 0.8.1-incubating + 3e1f78c Sun Dec 8 21:34:12 2013 -0800 + Merge pull request #195 from dhardy92/fix_DebScriptPackage + [[Deb] fix package of Spark classes adding org.apache prefix in scripts embeded in .deb] + c14f373 Sat Dec 7 22:35:31 2013 -0800 Merge pull request #241 from pwendell/master [Update broken links and add HDP 2.0 version string] From c88a9916a183e7a57b53537531620bbde6d8869a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 21:46:33 -0800 Subject: [PATCH 71/90] [maven-release-plugin] prepare release v0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7e9b201d3546d..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 44acbd9caf20a..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1264bc4933cfe..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 2ef2785e17fd3..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a19d8b3b450c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 83f9b23e12de2..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index d0e4428908056..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 86d2a0c050436..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 320d5326f3819..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d9168e33f6084..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From af7498870a4eed0f6e4b9fe37cc69edc022d0d8c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 8 Dec 2013 21:46:45 -0800 Subject: [PATCH 72/90] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..361a1d8c1621c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..2be372da59d5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..90ac301261581 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..d12e3abf2e89b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..bd6a1c35f1a8c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..53f576c54e81d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..6f14fe87ae650 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..7a59d89819dd1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..910401691f0a8 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..66e492e76f873 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..002c731851e52 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml From 1e8b0449563b36860d9e1edf352a61e103879afe Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 16:59:20 -0800 Subject: [PATCH 73/90] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit af7498870a4eed0f6e4b9fe37cc69edc022d0d8c. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 361a1d8c1621c..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 2be372da59d5f..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 90ac301261581..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d12e3abf2e89b..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bd6a1c35f1a8c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 53f576c54e81d..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6f14fe87ae650..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 7a59d89819dd1..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 910401691f0a8..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 66e492e76f873..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 002c731851e52..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From b6754abace2a91e7d1bfad966161d178b8a1f714 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 16:59:22 -0800 Subject: [PATCH 74/90] Revert "[maven-release-plugin] prepare release v0.8.1-incubating" This reverts commit c88a9916a183e7a57b53537531620bbde6d8869a. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..7e9b201d3546d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..44acbd9caf20a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..1264bc4933cfe 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..2ef2785e17fd3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..2a19d8b3b450c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..83f9b23e12de2 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..d0e4428908056 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..86d2a0c050436 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..320d5326f3819 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..21a6ba981262f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..d9168e33f6084 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml From 45672958fd02abb3c33e176a9faa157ec6d9c43e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 16:51:36 -0800 Subject: [PATCH 75/90] Merge pull request #246 from pwendell/master Add missing license headers I found this when doing further audits on the 0.8.1 release candidate. (cherry picked from commit 6169fe14a140146602fb07cfcd13eee6efad98f9) Signed-off-by: Patrick Wendell --- .../spark/storage/StoragePerfTester.scala | 17 +++++++++++++++++ .../deploy/worker/ExecutorRunnerTest.scala | 17 +++++++++++++++++ .../spark/storage/DiskBlockManagerSuite.scala | 17 +++++++++++++++++ .../util/collection/OpenHashMapSuite.scala | 17 +++++++++++++++++ .../util/collection/OpenHashSetSuite.scala | 17 +++++++++++++++++ .../PrimitiveKeyOpenHashSetSuite.scala | 17 +++++++++++++++++ python/test_support/userlibrary.py | 17 +++++++++++++++++ 7 files changed, 119 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala index 68893a2bf49b1..369a2e536b068 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.storage import java.util.concurrent.atomic.AtomicLong diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 8f0954122b322..4cb4ddc9cd66a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.deploy.worker import java.io.File diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index ef4c4c0f143f9..070982e798f34 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.storage import java.io.{FileWriter, File} diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index ca3f684668d60..80ee59f26f49e 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.util.collection import scala.collection.mutable.HashSet diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala index 4e11e8a628b44..5ad7ea74f186f 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.util.collection import org.scalatest.FunSuite diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala index dfd6aed2c4bcc..0342195161dc0 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.spark.util.collection import scala.collection.mutable.HashSet diff --git a/python/test_support/userlibrary.py b/python/test_support/userlibrary.py index 5bb6f5009ffbb..8e4a6292bc17c 100755 --- a/python/test_support/userlibrary.py +++ b/python/test_support/userlibrary.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. +# + """ Used to test shipping of code depenencies with SparkContext.addPyFile(). """ From 7e5564cc788c33a3048914d61e90eff88b1a3903 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 18:06:54 -0800 Subject: [PATCH 76/90] [maven-release-plugin] prepare release v0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7e9b201d3546d..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 44acbd9caf20a..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1264bc4933cfe..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 2ef2785e17fd3..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a19d8b3b450c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 83f9b23e12de2..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index d0e4428908056..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 86d2a0c050436..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 320d5326f3819..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d9168e33f6084..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 216b473df0d6912f4905204402cfe04568e0560c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 18:07:00 -0800 Subject: [PATCH 77/90] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..361a1d8c1621c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..2be372da59d5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..90ac301261581 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..d12e3abf2e89b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..bd6a1c35f1a8c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..53f576c54e81d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..6f14fe87ae650 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..7a59d89819dd1 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..910401691f0a8 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..66e492e76f873 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..002c731851e52 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml From e468f81164887a612377caedb398d102b3acdf50 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Tue, 10 Dec 2013 13:30:57 +0800 Subject: [PATCH 78/90] Fix POM file for mvn assembly on hadoop 2.2 Yarn --- assembly/pom.xml | 28 ++++++++++ examples/pom.xml | 17 ++++++ new-yarn/pom.xml | 18 +++++- pom.xml | 140 +++++++++++++++++++++++++++++++++++++++++++++++ repl-bin/pom.xml | 12 ++++ repl/pom.xml | 12 ++++ yarn/pom.xml | 17 ++++++ 7 files changed, 243 insertions(+), 1 deletion(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 361a1d8c1621c..e7d923c5e7d0b 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -39,6 +39,23 @@ + + ${akka.group} + akka-actor + + + ${akka.group} + akka-remote + + + ${akka.group} + akka-slf4j + + + ${akka.group} + akka-zeromq + + org.apache.spark spark-core_2.9.3 @@ -133,6 +150,17 @@ + + new-yarn + + + org.apache.spark + spark-yarn_2.9.3 + ${project.version} + + + + bigtop-dist + + + com.typesafe.akka + akka-actor + + + com.typesafe.akka + akka-remote + + + com.typesafe.akka + akka-slf4j + + + com.typesafe.akka + akka-zeromq + + + + + org.apache.spark + spark-bagel_2.9.3 + ${project.version} + + + com.typesafe.akka + akka-actor + + + com.typesafe.akka + akka-remote + + + com.typesafe.akka + akka-slf4j + + + com.typesafe.akka + akka-zeromq + + + + + org.apache.spark + spark-mllib_2.9.3 + ${project.version} + + + com.typesafe.akka + akka-actor + + + com.typesafe.akka + akka-remote + + + com.typesafe.akka + akka-slf4j + + + com.typesafe.akka + akka-zeromq + + + + + org.apache.spark + spark-repl_2.9.3 + ${project.version} + + + com.typesafe.akka + akka-actor + + + com.typesafe.akka + akka-remote + + + com.typesafe.akka + akka-slf4j + + + com.typesafe.akka + akka-zeromq + + + + + org.apache.spark + spark-streaming_2.9.3 + ${project.version} + + + com.typesafe.akka + akka-actor + + + com.typesafe.akka + akka-remote + + + com.typesafe.akka + akka-slf4j + + + com.typesafe.akka + akka-zeromq + + + + + org.apache.spark + spark-yarn_2.9.3 + ${project.version} + + + com.typesafe.akka + akka-actor + + + com.typesafe.akka + akka-remote + + + com.typesafe.akka + akka-slf4j + + + com.typesafe.akka + akka-zeromq + + + + org.eclipse.jetty jetty-server diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6f14fe87ae650..d6aa47a8f869d 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -38,6 +38,18 @@ + + ${akka.group} + akka-actor + + + ${akka.group} + akka-remote + + + ${akka.group} + akka-slf4j + org.apache.spark spark-core_2.9.3 diff --git a/repl/pom.xml b/repl/pom.xml index 7a59d89819dd1..b7dbff7ff3027 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -37,6 +37,18 @@ + + ${akka.group} + akka-actor + + + ${akka.group} + akka-remote + + + ${akka.group} + akka-slf4j + org.apache.spark spark-core_2.9.3 diff --git a/yarn/pom.xml b/yarn/pom.xml index 002c731851e52..04934aa8b8297 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -31,6 +31,23 @@ http://spark.incubator.apache.org/ + + ${akka.group} + akka-actor + + + ${akka.group} + akka-remote + + + ${akka.group} + akka-slf4j + + + ${akka.group} + akka-zeromq + + org.apache.spark spark-core_2.9.3 From 52c0890ebe5aa5cdb88bd1359717c31bccddd7c0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 23:08:33 -0800 Subject: [PATCH 79/90] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 216b473df0d6912f4905204402cfe04568e0560c. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 361a1d8c1621c..278931283a2f9 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 2be372da59d5f..8ec6acbdc92c1 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 90ac301261581..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index d12e3abf2e89b..bc9b7924033a7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bd6a1c35f1a8c..4e36d4c989d16 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 53f576c54e81d..4abddcc1f36d3 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6f14fe87ae650..dc4a21ab6aca4 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 7a59d89819dd1..20c6b87238953 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 910401691f0a8..834f155590a3a 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 66e492e76f873..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 002c731851e52..bcb8baff409a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From d61494588a72fb37b8766385c559ddf2acb7b00b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 23:08:44 -0800 Subject: [PATCH 80/90] Revert "[maven-release-plugin] prepare release v0.8.1-incubating" This reverts commit 7e5564cc788c33a3048914d61e90eff88b1a3903. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 278931283a2f9..7e9b201d3546d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 8ec6acbdc92c1..44acbd9caf20a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..1264bc4933cfe 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index bc9b7924033a7..2ef2785e17fd3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4e36d4c989d16..2a19d8b3b450c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4abddcc1f36d3..83f9b23e12de2 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index dc4a21ab6aca4..d0e4428908056 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 20c6b87238953..86d2a0c050436 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 834f155590a3a..320d5326f3819 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..21a6ba981262f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index bcb8baff409a2..d9168e33f6084 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml From 81293289ee7dfa4db0c5b4ec89faf4fa72d3ba10 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 23:13:15 -0800 Subject: [PATCH 81/90] Updating CHANGES and one fix from last merge --- CHANGES.txt | 4 ++++ new-yarn/pom.xml | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/CHANGES.txt b/CHANGES.txt index ce0aeab524505..efafab2745748 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -2,6 +2,10 @@ Spark Change Log Release 0.8.1-incubating + d03589d Mon Dec 9 23:10:00 2013 -0800 + Merge pull request #248 from colorant/branch-0.8 + [Fix POM file for mvn assembly on hadoop 2.2 Yarn] + 3e1f78c Sun Dec 8 21:34:12 2013 -0800 Merge pull request #195 from dhardy92/fix_DebScriptPackage [[Deb] fix package of Spark classes adding org.apache prefix in scripts embeded in .deb] diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml index 3310f04022e15..556fd378a4250 100644 --- a/new-yarn/pom.xml +++ b/new-yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating-SNAPSHOT ../pom.xml From 22fcb787a7e78ac6943c2fbcc43235c83bd1bfac Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Dec 2013 00:02:27 -0800 Subject: [PATCH 82/90] Add missing dependencies --- bagel/pom.xml | 12 ++++++++++++ mllib/pom.xml | 13 +++++++++++++ streaming/pom.xml | 12 ++++++++++++ yarn/pom.xml | 1 - 4 files changed, 37 insertions(+), 1 deletion(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 44acbd9caf20a..99e797ecdea87 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -32,6 +32,18 @@ http://spark.incubator.apache.org/ + + ${akka.group} + akka-actor + + + ${akka.group} + akka-remote + + + ${akka.group} + akka-slf4j + org.apache.spark spark-core_2.9.3 diff --git a/mllib/pom.xml b/mllib/pom.xml index 2a19d8b3b450c..5b09d07f51ab9 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -32,6 +32,19 @@ http://spark.incubator.apache.org/ + + ${akka.group} + akka-actor + + + ${akka.group} + akka-remote + + + ${akka.group} + akka-slf4j + + org.apache.spark spark-core_2.9.3 diff --git a/streaming/pom.xml b/streaming/pom.xml index 320d5326f3819..498e035ea8ab2 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -40,6 +40,18 @@ + + ${akka.group} + akka-actor + + + ${akka.group} + akka-remote + + + ${akka.group} + akka-slf4j + org.apache.spark spark-core_2.9.3 diff --git a/yarn/pom.xml b/yarn/pom.xml index c190cdc5d1f7a..556fd378a4250 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -47,7 +47,6 @@ ${akka.group} akka-zeromq - org.apache.spark spark-core_2.9.3 From c45a267a0d181d6641827ac4980dbeeace21ec4e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Dec 2013 00:08:02 -0800 Subject: [PATCH 83/90] Small fix from prior commit --- mllib/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index 5b09d07f51ab9..0a582277c44bf 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -44,7 +44,6 @@ ${akka.group} akka-slf4j - org.apache.spark spark-core_2.9.3 From b0e50f8548efc8857826bf138d2cf04029f70337 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Dec 2013 13:01:26 -0800 Subject: [PATCH 84/90] Merge pull request #250 from pwendell/master README incorrectly suggests build sources spark-env.sh This is misleading because the build doesn't source that file. IMO it's better to force people to specify build environment variables on the command line always, like we do in every example, so I'm just removing this doc. (cherry picked from commit d2efe13574090e93c600adeacc7f6356bc196e6c) Signed-off-by: Patrick Wendell --- README.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/README.md b/README.md index 5e3a65f3401c7..fc572d57e94c4 100644 --- a/README.md +++ b/README.md @@ -67,9 +67,6 @@ with YARN, also set `SPARK_YARN=true`: # Apache Hadoop 2.2.0 with YARN $ SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt assembly -For convenience, these variables may also be set through the `conf/spark-env.sh` file -described below. - When developing a Spark application, specify the Hadoop version by adding the "hadoop-client" artifact to your project's dependencies. For example, if you're using Hadoop 1.2.1 and build your application using SBT, add this entry to From 909a9e4d11eccea03980a8ed7ba7f9f27c68e33a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Dec 2013 13:22:11 -0800 Subject: [PATCH 85/90] [maven-release-plugin] prepare release v0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 75adf3cf060fc..1a760036c4d84 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 99e797ecdea87..98fca78b9a7e2 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1264bc4933cfe..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 1a162477e2b39..a6dc3200bffa2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 0a582277c44bf..40e697eb5de2a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 3b54fb4bd1e3d..77686ca691aeb 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 47920ac0102bc..2e3e3cc8887ca 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 12cef253b1c3c..9b6c7a83fb843 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 498e035ea8ab2..0f55f4b23ca97 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 556fd378a4250..d15404cf2e670 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 7ca53692c821ead3a50520fe51e229d9fc1f856d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Dec 2013 13:22:17 -0800 Subject: [PATCH 86/90] [maven-release-plugin] prepare for next development iteration --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 1a760036c4d84..e7d923c5e7d0b 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 98fca78b9a7e2..d72d9d9b0be06 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..90ac301261581 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index a6dc3200bffa2..5683d40df79fc 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 40e697eb5de2a..4b3e3dc3e34f9 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 77686ca691aeb..dbff1be8dc58b 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 2e3e3cc8887ca..d6aa47a8f869d 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 9b6c7a83fb843..b7dbff7ff3027 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 0f55f4b23ca97..6b3bd30bb80d4 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..66e492e76f873 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d15404cf2e670..3310f04022e15 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.2-incubating-SNAPSHOT ../pom.xml From d101dfe6cb4c20d4f559da769a234e4196aac98c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Dec 2013 14:22:31 -0800 Subject: [PATCH 87/90] Revert "[maven-release-plugin] prepare for next development iteration" This reverts commit 7ca53692c821ead3a50520fe51e229d9fc1f856d. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index e7d923c5e7d0b..1a760036c4d84 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index d72d9d9b0be06..98fca78b9a7e2 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 90ac301261581..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 5683d40df79fc..a6dc3200bffa2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 4b3e3dc3e34f9..40e697eb5de2a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index dbff1be8dc58b..77686ca691aeb 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index d6aa47a8f869d..2e3e3cc8887ca 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index b7dbff7ff3027..9b6c7a83fb843 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 6b3bd30bb80d4..0f55f4b23ca97 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 66e492e76f873..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 3310f04022e15..d15404cf2e670 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.2-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From 9415d2daaa6473ad2dd332d23ae2565c56b3b784 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Dec 2013 14:22:34 -0800 Subject: [PATCH 88/90] Revert "[maven-release-plugin] prepare release v0.8.1-incubating" This reverts commit 909a9e4d11eccea03980a8ed7ba7f9f27c68e33a. --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 1a760036c4d84..75adf3cf060fc 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 98fca78b9a7e2..99e797ecdea87 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index dbef7ba191c84..1264bc4933cfe 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index a6dc3200bffa2..1a162477e2b39 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 40e697eb5de2a..0a582277c44bf 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 77686ca691aeb..3b54fb4bd1e3d 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - v0.8.1-incubating + HEAD diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 2e3e3cc8887ca..47920ac0102bc 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 9b6c7a83fb843..12cef253b1c3c 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 0f55f4b23ca97..498e035ea8ab2 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c3bb2940ec399..21a6ba981262f 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index d15404cf2e670..556fd378a4250 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating + 0.8.1-incubating-SNAPSHOT ../pom.xml From 628ca8559c797007a62842290544ab89082564cd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Dec 2013 14:24:26 -0800 Subject: [PATCH 89/90] Small bug fix in YARN build patch --- new-yarn/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/new-yarn/pom.xml b/new-yarn/pom.xml index 556fd378a4250..d15404cf2e670 100644 --- a/new-yarn/pom.xml +++ b/new-yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml From b87d31dd8eb4b4e47c0138e9242d0dd6922c8c4e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 10 Dec 2013 14:35:56 -0800 Subject: [PATCH 90/90] [maven-release-plugin] prepare release v0.8.1-incubating --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 11 files changed, 12 insertions(+), 12 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 75adf3cf060fc..1a760036c4d84 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 99e797ecdea87..98fca78b9a7e2 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 1264bc4933cfe..dbef7ba191c84 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 1a162477e2b39..a6dc3200bffa2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 0a582277c44bf..40e697eb5de2a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/pom.xml b/pom.xml index 3b54fb4bd1e3d..77686ca691aeb 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating pom Spark Project Parent POM http://spark.incubator.apache.org/ @@ -40,7 +40,7 @@ scm:git:git@github.com:apache/incubator-spark.git scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git scm:git:git@github.com:apache/incubator-spark.git - HEAD + v0.8.1-incubating diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 47920ac0102bc..2e3e3cc8887ca 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/repl/pom.xml b/repl/pom.xml index 12cef253b1c3c..9b6c7a83fb843 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 498e035ea8ab2..0f55f4b23ca97 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 21a6ba981262f..c3bb2940ec399 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 556fd378a4250..d15404cf2e670 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.1-incubating-SNAPSHOT + 0.8.1-incubating ../pom.xml