Fixed compile time warnings and formatting post merge.

This commit is contained in:
Prashant Sharma 2013-11-26 14:26:23 +05:30
parent 44fd30d3fb
commit d092a8cc6a
5 changed files with 79 additions and 73 deletions

View file

@ -17,9 +17,7 @@
package org.apache.spark.deploy.master
private[spark] object RecoveryState
extends Enumeration("STANDBY", "ALIVE", "RECOVERING", "COMPLETING_RECOVERY") {
private[spark] object RecoveryState extends Enumeration {
type MasterState = Value
val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value

View file

@ -553,7 +553,7 @@ abstract class RDD[T: ClassTag](
(f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, false)
def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]
def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean)
(f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, preservesPartitioning)
@ -563,7 +563,7 @@ abstract class RDD[T: ClassTag](
(f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, false)
def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]
def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D], preservesPartitioning: Boolean)
(f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, preservesPartitioning)

View file

@ -55,8 +55,7 @@ class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, clea
}
}
object MetadataCleanerType extends Enumeration("MapOutputTracker", "SparkContext", "HttpBroadcast", "DagScheduler", "ResultTask",
"ShuffleMapTask", "BlockManager", "DiskBlockManager", "BroadcastVars") {
object MetadataCleanerType extends Enumeration {
val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK,
SHUFFLE_MAP_TASK, BLOCK_MANAGER, SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value

View file

@ -39,7 +39,7 @@ class BlockIdSuite extends FunSuite {
fail()
} catch {
case e: IllegalStateException => // OK
case _ => fail()
case _: Throwable => fail()
}
}

View file

@ -79,7 +79,8 @@ class ReplSuite extends FunSuite {
}
test("simple foreach with accumulator") {
val output = runInterpreter("local", """
val output = runInterpreter("local",
"""
|val accum = sc.accumulator(0)
|sc.parallelize(1 to 10).foreach(x => accum += x)
|accum.value
@ -90,7 +91,8 @@ class ReplSuite extends FunSuite {
}
test("external vars") {
val output = runInterpreter("local", """
val output = runInterpreter("local",
"""
|var v = 7
|sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
|v = 10
@ -103,7 +105,8 @@ class ReplSuite extends FunSuite {
}
test("external classes") {
val output = runInterpreter("local", """
val output = runInterpreter("local",
"""
|class C {
|def foo = 5
|}
@ -115,7 +118,8 @@ class ReplSuite extends FunSuite {
}
test("external functions") {
val output = runInterpreter("local", """
val output = runInterpreter("local",
"""
|def double(x: Int) = x + x
|sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_)
""".stripMargin)
@ -125,7 +129,8 @@ class ReplSuite extends FunSuite {
}
test("external functions that access vars") {
val output = runInterpreter("local", """
val output = runInterpreter("local",
"""
|var v = 7
|def getV() = v
|sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
@ -142,7 +147,8 @@ class ReplSuite extends FunSuite {
// Test that the value that a broadcast var had when it was created is used,
// even if that variable is then modified in the driver program
// TODO: This doesn't actually work for arrays when we run in local mode!
val output = runInterpreter("local", """
val output = runInterpreter("local",
"""
|var array = new Array[Int](5)
|val broadcastArray = sc.broadcast(array)
|sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
@ -162,7 +168,8 @@ class ReplSuite extends FunSuite {
out.write("What's up?\n")
out.write("Goodbye\n")
out.close()
val output = runInterpreter("local", """
val output = runInterpreter("local",
"""
|var file = sc.textFile("%s/input").cache()
|file.count()
|file.count()
@ -176,7 +183,8 @@ class ReplSuite extends FunSuite {
}
test("local-cluster mode") {
val output = runInterpreter("local-cluster[1,1,512]", """
val output = runInterpreter("local-cluster[1,1,512]",
"""
|var v = 7
|def getV() = v
|sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
@ -198,7 +206,8 @@ class ReplSuite extends FunSuite {
if (System.getenv("MESOS_NATIVE_LIBRARY") != null) {
test("running on Mesos") {
val output = runInterpreter("localquiet", """
val output = runInterpreter("localquiet",
"""
|var v = 7
|def getV() = v
|sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)