From c9bc8af3d17d153bc182dcddc6611b1fa87ffbbf Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 3 Sep 2013 15:25:20 -0700 Subject: [PATCH 01/34] Removed repetative import; fixes hidden definition compiler warning. --- core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index adc971050e..0be00ef996 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -25,7 +25,6 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import scala.collection.parallel.mutable import org.apache.spark._ -import org.apache.spark.rdd.CoalescedRDDPartition class RDDSuite extends FunSuite with SharedSparkContext { From 6919a28d51c416ff4bb647b03eae2070cf87f039 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 17 May 2013 17:10:47 -0700 Subject: [PATCH 02/34] Construct shell commands as sequences for safety and composability --- ec2/spark_ec2.py | 45 ++++++++++++++++++++++++++++++++++----------- 1 file changed, 34 insertions(+), 11 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 932e70db96..75dd0ffa61 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -23,6 +23,7 @@ from __future__ import with_statement import logging import os +import pipes import random import shutil import subprocess @@ -536,18 +537,41 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): dest.write(text) dest.close() # rsync the whole directory over to the master machine - command = (("rsync -rv -e 'ssh -o StrictHostKeyChecking=no -i %s' " + - "'%s/' '%s@%s:/'") % (opts.identity_file, tmp_dir, opts.user, active_master)) - subprocess.check_call(command, shell=True) + command = [ + 'rsync', '-rv', + '-e', stringify_command(ssh_command(opts)), + "%s/" % tmp_dir, + "%s@%s:/" % (opts.user, active_master) + ] + subprocess.check_call(command) # Remove the temp directory we created above shutil.rmtree(tmp_dir) +def stringify_command(parts): + if isinstance(parts, str): + return parts + else: + return ' '.join(map(pipes.quote, parts)) + + +def ssh_args(opts): + parts = ['-o', 'StrictHostKeyChecking=no', '-i', opts.identity_file] + return parts + + +def ssh_command(opts): + return ['ssh'] + ssh_args(opts) + + +def scp_command(opts): + return ['scp', '-q'] + ssh_args(opts) + + # Copy a file to a given host through scp, throwing an exception if scp fails def scp(host, opts, local_file, dest_file): subprocess.check_call( - "scp -q -o StrictHostKeyChecking=no -i %s '%s' '%s@%s:%s'" % - (opts.identity_file, local_file, opts.user, host, dest_file), shell=True) + scp_command(opts) + [local_file, "%s@%s:%s" % (opts.user, host, dest_file)]) # Run a command on a host through ssh, retrying up to two times @@ -557,8 +581,7 @@ def ssh(host, opts, command): while True: try: return subprocess.check_call( - "ssh -t -o StrictHostKeyChecking=no -i %s %s@%s '%s'" % - (opts.identity_file, opts.user, host, command), shell=True) + ssh_command(opts) + ['-t', '%s@%s' % (opts.user, host), stringify_command(command)]) except subprocess.CalledProcessError as e: if (tries > 2): raise e @@ -670,11 +693,11 @@ def main(): conn, opts, cluster_name) master = master_nodes[0].public_dns_name print "Logging into master " + master + "..." - proxy_opt = "" + proxy_opt = [] if opts.proxy_port != None: - proxy_opt = "-D " + opts.proxy_port - subprocess.check_call("ssh -o StrictHostKeyChecking=no -i %s %s %s@%s" % - (opts.identity_file, proxy_opt, opts.user, master), shell=True) + proxy_opt = ['-D', opts.proxy_port] + subprocess.check_call( + ssh_command(opts) + proxy_opt + ['-t', "%s@%s" % (opts.user, master)]) elif action == "get-master": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) From b98572c70ad3932381a55f23f82600d7e435d2eb Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 3 Jul 2013 16:57:22 -0700 Subject: [PATCH 03/34] Generate new SSH key for the cluster, make "--identity-file" optional --- ec2/spark_ec2.py | 58 ++++++++++++++++++++++++++++++------------------ 1 file changed, 37 insertions(+), 21 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 75dd0ffa61..0858b126c5 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -104,11 +104,7 @@ def parse_args(): parser.print_help() sys.exit(1) (action, cluster_name) = args - if opts.identity_file == None and action in ['launch', 'login', 'start']: - print >> stderr, ("ERROR: The -i or --identity-file argument is " + - "required for " + action) - sys.exit(1) - + # Boto config check # http://boto.cloudhackers.com/en/latest/boto_config_tut.html home_dir = os.getenv('HOME') @@ -392,10 +388,18 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): master = master_nodes[0].public_dns_name if deploy_ssh_key: - print "Copying SSH key %s to master..." % opts.identity_file - ssh(master, opts, 'mkdir -p ~/.ssh') - scp(master, opts, opts.identity_file, '~/.ssh/id_rsa') - ssh(master, opts, 'chmod 600 ~/.ssh/id_rsa') + print "Generating cluster's SSH key on master..." + key_setup = """ + [ -f ~/.ssh/id_rsa ] || + (ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa && + cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys) + """ + ssh(master, opts, key_setup) + dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh']) + print "Transferring cluster's SSH key to slaves..." + for slave in slave_nodes: + print slave.public_dns_name + ssh_write(slave.public_dns_name, opts, ['tar', 'x'], dot_ssh_tar) modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs', 'mapreduce', 'spark-standalone'] @@ -556,7 +560,9 @@ def stringify_command(parts): def ssh_args(opts): - parts = ['-o', 'StrictHostKeyChecking=no', '-i', opts.identity_file] + parts = ['-o', 'StrictHostKeyChecking=no'] + if opts.identity_file is not None: + parts += ['-i', opts.identity_file] return parts @@ -564,16 +570,6 @@ def ssh_command(opts): return ['ssh'] + ssh_args(opts) -def scp_command(opts): - return ['scp', '-q'] + ssh_args(opts) - - -# Copy a file to a given host through scp, throwing an exception if scp fails -def scp(host, opts, local_file, dest_file): - subprocess.check_call( - scp_command(opts) + [local_file, "%s@%s:%s" % (opts.user, host, dest_file)]) - - # Run a command on a host through ssh, retrying up to two times # and then throwing an exception if ssh continues to fail. def ssh(host, opts, command): @@ -585,13 +581,33 @@ def ssh(host, opts, command): except subprocess.CalledProcessError as e: if (tries > 2): raise e - print "Couldn't connect to host {0}, waiting 30 seconds".format(e) + print "Error connecting to host, sleeping 30: {0}".format(e) time.sleep(30) tries = tries + 1 +def ssh_read(host, opts, command): + return subprocess.check_output( + ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)]) +def ssh_write(host, opts, command, input): + tries = 0 + while True: + proc = subprocess.Popen( + ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)], + stdin=subprocess.PIPE) + proc.stdin.write(input) + proc.stdin.close() + if proc.wait() == 0: + break + elif (tries > 2): + raise RuntimeError("ssh_write error %s" % proc.returncode) + else: + print "Error connecting to host, sleeping 30" + time.sleep(30) + tries = tries + 1 + # Gets a list of zones to launch instances in def get_zones(conn, opts): From 293c758cc00b51786293158da4870c669f416a7f Mon Sep 17 00:00:00 2001 From: Mike Date: Tue, 10 Sep 2013 00:24:35 -0700 Subject: [PATCH 04/34] Remove MemoryStore$Entry.dropPending, unused as of 42e0a68082. --- core/src/main/scala/org/apache/spark/storage/MemoryStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 3b3b2342fa..c465a9ad6c 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -30,7 +30,7 @@ import org.apache.spark.util.{SizeEstimator, Utils} private class MemoryStore(blockManager: BlockManager, maxMemory: Long) extends BlockStore(blockManager) { - case class Entry(value: Any, size: Long, deserialized: Boolean, var dropPending: Boolean = false) + case class Entry(value: Any, size: Long, deserialized: Boolean) private val entries = new LinkedHashMap[String, Entry](32, 0.75f, true) private var currentMemory = 0L From e86d1d4a52147fe52feeda74ca3558f6bc109285 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 11 Sep 2013 14:59:42 -0700 Subject: [PATCH 05/34] Clarify error messages on SSH failure --- ec2/spark_ec2.py | 27 +++++++++++++++++++++------ 1 file changed, 21 insertions(+), 6 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0858b126c5..f4babba9b9 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -37,6 +37,9 @@ import boto from boto.ec2.blockdevicemapping import BlockDeviceMapping, EBSBlockDeviceType from boto import ec2 +class UsageError(Exception): + pass + # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list" @@ -580,8 +583,12 @@ def ssh(host, opts, command): ssh_command(opts) + ['-t', '%s@%s' % (opts.user, host), stringify_command(command)]) except subprocess.CalledProcessError as e: if (tries > 2): - raise e - print "Error connecting to host, sleeping 30: {0}".format(e) + # If this was an ssh failure, provide the user with hints. + if e.returncode == 255: + raise UsageError("Failed to SSH to remote host {0}.\nPlease check that you have provided the correct --identity-file and --key-pair parameters and try again.".format(host)) + else: + raise e + print >> stderr, "Error executing remote command, retrying after 30 seconds: {0}".format(e) time.sleep(30) tries = tries + 1 @@ -599,12 +606,13 @@ def ssh_write(host, opts, command, input): stdin=subprocess.PIPE) proc.stdin.write(input) proc.stdin.close() - if proc.wait() == 0: + status = proc.wait() + if status == 0: break elif (tries > 2): - raise RuntimeError("ssh_write error %s" % proc.returncode) + raise RuntimeError("ssh_write failed with error %s" % proc.returncode) else: - print "Error connecting to host, sleeping 30" + print >> stderr, "Error {0} while executing remote command, retrying after 30 seconds".format(status) time.sleep(30) tries = tries + 1 @@ -626,7 +634,7 @@ def get_partition(total, num_partitions, current_partitions): return num_slaves_this_zone -def main(): +def real_main(): (opts, action, cluster_name) = parse_args() try: conn = ec2.connect_to_region(opts.region) @@ -755,6 +763,13 @@ def main(): sys.exit(1) +def main(): + try: + real_main() + except UsageError, e: + print >> stderr, "\nError:\n", e + + if __name__ == "__main__": logging.basicConfig() main() From d34672f6684d2c14fc5db58335370ef9ba84375e Mon Sep 17 00:00:00 2001 From: Mike Date: Wed, 11 Sep 2013 18:01:19 -0700 Subject: [PATCH 06/34] Set currentMemory to 0 in clear(). Remove unnecessary entries.get() call. --- .../src/main/scala/org/apache/spark/storage/MemoryStore.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index c465a9ad6c..4344f851d9 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -110,9 +110,8 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) override def remove(blockId: String): Boolean = { entries.synchronized { - val entry = entries.get(blockId) + val entry = entries.remove(blockId) if (entry != null) { - entries.remove(blockId) currentMemory -= entry.size logInfo("Block %s of size %d dropped from memory (free %d)".format( blockId, entry.size, freeMemory)) @@ -126,6 +125,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) override def clear() { entries.synchronized { entries.clear() + currentMemory = 0 } logInfo("MemoryStore cleared") } From 74f710f6cda31c1489e8f0cc130021ce4e9e60c6 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Wed, 11 Sep 2013 22:35:58 -0700 Subject: [PATCH 07/34] Start of working on SPARK-615 --- .../scala/org/apache/spark/api/java/JavaRDDLike.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 7e6e691f11..9ad175ec19 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -67,6 +67,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def map[R](f: JFunction[T, R]): JavaRDD[R] = new JavaRDD(rdd.map(f)(f.returnType()))(f.returnType()) + /** + * Return a new RDD by applying a function to each partition of this RDD, while tracking the index + * of the original partition. + */ + def mapPartitionsWithIndex(f: JFunction2[Int, T, R], + preservesPartitioning: Boolean = false): JavaRDD[R] = + new JavaRDD(MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)) + /** * Return a new RDD by applying a function to all elements of this RDD. */ From bfcddf4700023f53d5eed92ef8ef75c072af3ced Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 14 Sep 2013 15:53:42 -0700 Subject: [PATCH 08/34] Make mapPartitionsWithIndex work with JavaRDD's --- .../main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 9ad175ec19..264c4bc3de 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -71,9 +71,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithIndex(f: JFunction2[Int, T, R], + def mapPartitionsWithIndex[R: ClassManifest](f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = - new JavaRDD(MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)) + new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), + preservesPartitioning)) /** * Return a new RDD by applying a function to all elements of this RDD. From 68068977b85d2355223e21ebf4e546a13f0a8585 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 14 Sep 2013 20:51:11 -0700 Subject: [PATCH 09/34] Fix build on ubuntu --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 77e211ce03..eb5a89394b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -81,7 +81,7 @@ object SparkBuild extends Build { organization := "org.apache.spark", version := "0.8.0-SNAPSHOT", scalaVersion := "2.9.3", - scalacOptions := Seq("-unchecked", "-optimize", "-deprecation", + scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-optimize", "-deprecation", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, From ffa5f8e11db26dd616e85b9d941de3590ca3643e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 18 Sep 2013 17:33:24 +0800 Subject: [PATCH 10/34] Fix issue when local properties pass from parent to child thread --- .../scala/org/apache/spark/SparkContext.scala | 6 ++- .../org/apache/spark/ThreadingSuite.scala | 38 ++++++++++++++++++- 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 72540c712a..3922e9a7fa 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -256,7 +256,9 @@ class SparkContext( private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack - private val localProperties = new ThreadLocal[Properties] + private val localProperties = new InheritableThreadLocal[Properties] { + override protected def childValue(parent: Properties): Properties = new Properties(parent) + } def initLocalProperties() { localProperties.set(new Properties()) @@ -273,6 +275,8 @@ class SparkContext( } } + def getLocalProperty(key: String): String = Option(localProperties.get).map(_.getProperty(key)).getOrElse(null) + /** Set a human readable description of the current job. */ def setJobDescription(value: String) { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value) diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index 69383ddfb8..331f79dba1 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -40,7 +40,7 @@ object ThreadingSuiteState { } class ThreadingSuite extends FunSuite with LocalSparkContext { - + test("accessing SparkContext form a different thread") { sc = new SparkContext("local", "test") val nums = sc.parallelize(1 to 10, 2) @@ -149,4 +149,40 @@ class ThreadingSuite extends FunSuite with LocalSparkContext { fail("One or more threads didn't see runningThreads = 4") } } + + test("set local properties in different thread") { + sc = new SparkContext("local", "test") + + val threads = (1 to 5).map{ i => + new Thread() { + override def run() { + sc.setLocalProperty("test", i.toString) + assert(sc.getLocalProperty("test") === i.toString) + } + } + } + + threads.foreach(_.start()) + + assert(sc.getLocalProperty("test") === null) + } + + test("set and get local properties in parent-children thread") { + sc = new SparkContext("local", "test") + sc.setLocalProperty("test", "parent") + + val threads = (1 to 5).map{ i => + new Thread() { + override def run() { + assert(sc.getLocalProperty("test") === "parent") + sc.setLocalProperty("test", i.toString) + assert(sc.getLocalProperty("test") === i.toString) + } + } + } + + threads.foreach(_.start()) + assert(sc.getLocalProperty("test") === "parent") + assert(sc.getLocalProperty("Foo") === null) + } } From 026dba6abaaf6314a79ce873bb38b73a9b7fd1a7 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Thu, 19 Sep 2013 22:05:23 -0700 Subject: [PATCH 11/34] After unit tests, clear port properties unconditionally In MapOutputTrackerSuite, the "remote fetch" test sets spark.driver.port and spark.hostPort, assuming that they will be cleared by LocalSparkContext. However, the test never sets sc, so it remains null, causing LocalSparkContext to skip clearing these properties. Subsequent tests therefore fail with java.net.BindException: "Address already in use". This commit makes LocalSparkContext clear the properties even if sc is null. --- .../scala/org/apache/spark/LocalSparkContext.scala | 10 +++++----- .../scala/org/apache/spark/SharedSparkContext.scala | 6 ++---- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 6ec124da9c..459e257d79 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -40,17 +40,17 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self } def resetSparkContext() = { - if (sc != null) { - LocalSparkContext.stop(sc) - sc = null - } + LocalSparkContext.stop(sc) + sc = null } } object LocalSparkContext { def stop(sc: SparkContext) { - sc.stop() + if (sc != null) { + sc.stop() + } // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala index 97cbca09bf..288aa14eeb 100644 --- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -33,10 +33,8 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => } override def afterAll() { - if (_sc != null) { - LocalSparkContext.stop(_sc) - _sc = null - } + LocalSparkContext.stop(_sc) + _sc = null super.afterAll() } } From 9524b943a4f01297b9c5582d436e8af37d786d5e Mon Sep 17 00:00:00 2001 From: Mike Date: Thu, 19 Sep 2013 23:31:35 -0700 Subject: [PATCH 12/34] Synchronize on "entries" the remaining update to "currentMemory". Make "currentMemory" @volatile, so that it's reads in ensureFreeSpace() are atomic and up-to-date--i.e., currentMemory can't increase while putLock is held (though it could decrease, which would only help ensureFreeSpace()). --- .../main/scala/org/apache/spark/storage/MemoryStore.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 4344f851d9..77a39c71ed 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -33,7 +33,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) case class Entry(value: Any, size: Long, deserialized: Boolean) private val entries = new LinkedHashMap[String, Entry](32, 0.75f, true) - private var currentMemory = 0L + @volatile private var currentMemory = 0L // Object used to ensure that only one thread is putting blocks and if necessary, dropping // blocks from the memory store. private val putLock = new Object() @@ -160,8 +160,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) putLock.synchronized { if (ensureFreeSpace(blockId, size)) { val entry = new Entry(value, size, deserialized) - entries.synchronized { entries.put(blockId, entry) } - currentMemory += size + entries.synchronized { + entries.put(blockId, entry) + currentMemory += size + } if (deserialized) { logInfo("Block %s stored as values to memory (estimated size %s, free %s)".format( blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) From 42571d30d0d518e69eecf468075e4c5a823a2ae8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 Sep 2013 17:09:53 -0700 Subject: [PATCH 13/34] Smarter take/limit implementation. --- .../main/scala/org/apache/spark/rdd/RDD.scala | 38 ++++++++++++++----- .../scala/org/apache/spark/rdd/RDDSuite.scala | 38 +++++++++++++++++++ 2 files changed, 66 insertions(+), 10 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 1082cbae3e..1893627ee2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -753,24 +753,42 @@ abstract class RDD[T: ClassManifest]( } /** - * Take the first num elements of the RDD. This currently scans the partitions *one by one*, so - * it will be slow if a lot of partitions are required. In that case, use collect() to get the - * whole RDD instead. + * Take the first num elements of the RDD. It works by first scanning one partition, and use the + * results from that partition to estimate the number of additional partitions needed to satisfy + * the limit. */ def take(num: Int): Array[T] = { if (num == 0) { return new Array[T](0) } + val buf = new ArrayBuffer[T] - var p = 0 - while (buf.size < num && p < partitions.size) { + val totalParts = this.partitions.length + var partsScanned = 0 + while (buf.size < num && partsScanned < totalParts) { + // The number of partitions to try in this iteration. It is ok for this number to be + // greater than totalParts because we actually cap it at totalParts in runJob. + var numPartsToTry = 1 + if (partsScanned > 0) { + // If we didn't find any rows after the first iteration, just try all partitions next. + // Otherwise, interpolate the number of partitions we need to try, but overestimate it + // by 50%. + if (buf.size == 0) { + numPartsToTry = totalParts - 1 + } else { + numPartsToTry = (1.5 * num * partsScanned / buf.size).toInt + } + } + numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions + val left = num - buf.size - val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, Array(p), true) - buf ++= res(0) - if (buf.size == num) - return buf.toArray - p += 1 + val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) + val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p, allowLocal = true) + + res.foreach(buf ++= _.take(num - buf.size)) + partsScanned += numPartsToTry } + return buf.toArray } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index c1df5e151e..63adf1cda5 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -321,6 +321,44 @@ class RDDSuite extends FunSuite with SharedSparkContext { for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) } + test("take") { + var nums = sc.makeRDD(Range(1, 1000), 1) + assert(nums.take(0).size === 0) + assert(nums.take(1) === Array(1)) + assert(nums.take(3) === Array(1, 2, 3)) + assert(nums.take(500) === (1 to 500).toArray) + assert(nums.take(501) === (1 to 501).toArray) + assert(nums.take(999) === (1 to 999).toArray) + assert(nums.take(1000) === (1 to 999).toArray) + + nums = sc.makeRDD(Range(1, 1000), 2) + assert(nums.take(0).size === 0) + assert(nums.take(1) === Array(1)) + assert(nums.take(3) === Array(1, 2, 3)) + assert(nums.take(500) === (1 to 500).toArray) + assert(nums.take(501) === (1 to 501).toArray) + assert(nums.take(999) === (1 to 999).toArray) + assert(nums.take(1000) === (1 to 999).toArray) + + nums = sc.makeRDD(Range(1, 1000), 100) + assert(nums.take(0).size === 0) + assert(nums.take(1) === Array(1)) + assert(nums.take(3) === Array(1, 2, 3)) + assert(nums.take(500) === (1 to 500).toArray) + assert(nums.take(501) === (1 to 501).toArray) + assert(nums.take(999) === (1 to 999).toArray) + assert(nums.take(1000) === (1 to 999).toArray) + + nums = sc.makeRDD(Range(1, 1000), 1000) + assert(nums.take(0).size === 0) + assert(nums.take(1) === Array(1)) + assert(nums.take(3) === Array(1, 2, 3)) + assert(nums.take(500) === (1 to 500).toArray) + assert(nums.take(501) === (1 to 501).toArray) + assert(nums.take(999) === (1 to 999).toArray) + assert(nums.take(1000) === (1 to 999).toArray) + } + test("top with predefined ordering") { val nums = Array.range(1, 100000) val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) From 8933f9e98e8d39717477afa0bb7ffbc6872e05b9 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 20 Sep 2013 19:27:08 -0700 Subject: [PATCH 14/34] Add "org.apache." prefix to packages in spark-class Lacking this, the if/case statements never trigger on Spark 0.8.0+. --- spark-class | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/spark-class b/spark-class index 037abda3b7..e111ef6da7 100755 --- a/spark-class +++ b/spark-class @@ -37,7 +37,7 @@ fi # If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable # values for that; it doesn't need a lot -if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then +if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" # Do not overwrite SPARK_JAVA_OPTS environment variable in this script @@ -49,19 +49,19 @@ fi # Add java opts for master, worker, executor. The opts maybe null case "$1" in - 'spark.deploy.master.Master') + 'org.apache.spark.deploy.master.Master') OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS" ;; - 'spark.deploy.worker.Worker') + 'org.apache.spark.deploy.worker.Worker') OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS" ;; - 'spark.executor.StandaloneExecutorBackend') + 'org.apache.spark.executor.StandaloneExecutorBackend') OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" ;; - 'spark.executor.MesosExecutorBackend') + 'org.apache.spark.executor.MesosExecutorBackend') OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" ;; - 'spark.repl.Main') + 'org.apache.spark.repl.Main') OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS" ;; esac From aa0c29f74779bc5af70250c7481dbd7052ee39cf Mon Sep 17 00:00:00 2001 From: jerryshao Date: Sun, 22 Sep 2013 09:40:40 +0800 Subject: [PATCH 15/34] Add barrier for local properties unit test and fix some styles --- .../main/scala/org/apache/spark/SparkContext.scala | 3 ++- .../test/scala/org/apache/spark/ThreadingSuite.scala | 11 +++++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3922e9a7fa..6bab1f31d0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -275,7 +275,8 @@ class SparkContext( } } - def getLocalProperty(key: String): String = Option(localProperties.get).map(_.getProperty(key)).getOrElse(null) + def getLocalProperty(key: String): String = + Option(localProperties.get).map(_.getProperty(key)).getOrElse(null) /** Set a human readable description of the current job. */ def setJobDescription(value: String) { diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index 331f79dba1..75d6493e33 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -152,36 +152,43 @@ class ThreadingSuite extends FunSuite with LocalSparkContext { test("set local properties in different thread") { sc = new SparkContext("local", "test") + val sem = new Semaphore(0) - val threads = (1 to 5).map{ i => + val threads = (1 to 5).map { i => new Thread() { override def run() { sc.setLocalProperty("test", i.toString) assert(sc.getLocalProperty("test") === i.toString) + sem.release() } } } threads.foreach(_.start()) + sem.acquire(5) assert(sc.getLocalProperty("test") === null) } test("set and get local properties in parent-children thread") { sc = new SparkContext("local", "test") sc.setLocalProperty("test", "parent") + val sem = new Semaphore(0) - val threads = (1 to 5).map{ i => + val threads = (1 to 5).map { i => new Thread() { override def run() { assert(sc.getLocalProperty("test") === "parent") sc.setLocalProperty("test", i.toString) assert(sc.getLocalProperty("test") === i.toString) + sem.release() } } } threads.foreach(_.start()) + + sem.acquire(5) assert(sc.getLocalProperty("test") === "parent") assert(sc.getLocalProperty("Foo") === null) } From 5850f599dd93d2871e4e9777531cbbcae80a2475 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 12 Sep 2013 10:44:15 +0800 Subject: [PATCH 16/34] Refactor FairSchedulableBuilder: 1. Configuration can be read from classpath if not set explicitly. 2. Add missing close handler. --- .../cluster/SchedulableBuilder.scala | 92 +++++++++++-------- 1 file changed, 53 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala index f80823317b..f25924befa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import java.io.{File, FileInputStream, FileOutputStream, FileNotFoundException} +import java.io.{File, FileInputStream, FileOutputStream, FileNotFoundException, InputStream} import java.util.Properties import scala.xml.XML @@ -51,7 +51,8 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { - val schedulerAllocFile = System.getProperty("spark.scheduler.allocation.file") + val schedulerAllocFile = Option(System.getProperty("spark.scheduler.allocation.file")) + val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml" val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool" val DEFAULT_POOL_NAME = "default" val MINIMUM_SHARES_PROPERTY = "minShare" @@ -64,48 +65,26 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) val DEFAULT_WEIGHT = 1 override def buildPools() { - if (schedulerAllocFile != null) { - val file = new File(schedulerAllocFile) - if (file.exists()) { - val xml = XML.loadFile(file) - for (poolNode <- (xml \\ POOLS_PROPERTY)) { - - val poolName = (poolNode \ POOL_NAME_PROPERTY).text - var schedulingMode = DEFAULT_SCHEDULING_MODE - var minShare = DEFAULT_MINIMUM_SHARE - var weight = DEFAULT_WEIGHT - - val xmlSchedulingMode = (poolNode \ SCHEDULING_MODE_PROPERTY).text - if (xmlSchedulingMode != "") { - try { - schedulingMode = SchedulingMode.withName(xmlSchedulingMode) - } catch { - case e: Exception => logInfo("Error xml schedulingMode, using default schedulingMode") - } - } - - val xmlMinShare = (poolNode \ MINIMUM_SHARES_PROPERTY).text - if (xmlMinShare != "") { - minShare = xmlMinShare.toInt - } - - val xmlWeight = (poolNode \ WEIGHT_PROPERTY).text - if (xmlWeight != "") { - weight = xmlWeight.toInt - } - - val pool = new Pool(poolName, schedulingMode, minShare, weight) - rootPool.addSchedulable(pool) - logInfo("Created pool %s, schedulingMode: %s, minShare: %d, weight: %d".format( - poolName, schedulingMode, minShare, weight)) + var is: Option[InputStream] = None + try { + is = Option { + schedulerAllocFile map { f => + new FileInputStream(f) + } getOrElse { + getClass.getClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) } - } else { - throw new java.io.FileNotFoundException( - "Fair scheduler allocation file not found: " + schedulerAllocFile) } + + is foreach { i => buildFairSchedulerPool(i) } + } finally { + is.foreach(_.close) } // finally create "default" pool + buildDefaultPool() + } + + private def buildDefaultPool() { if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) { val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) @@ -115,6 +94,41 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) } } + private def buildFairSchedulerPool(is: InputStream) { + val xml = XML.load(is) + for (poolNode <- (xml \\ POOLS_PROPERTY)) { + + val poolName = (poolNode \ POOL_NAME_PROPERTY).text + var schedulingMode = DEFAULT_SCHEDULING_MODE + var minShare = DEFAULT_MINIMUM_SHARE + var weight = DEFAULT_WEIGHT + + val xmlSchedulingMode = (poolNode \ SCHEDULING_MODE_PROPERTY).text + if (xmlSchedulingMode != "") { + try { + schedulingMode = SchedulingMode.withName(xmlSchedulingMode) + } catch { + case e: Exception => logInfo("Error xml schedulingMode, using default schedulingMode") + } + } + + val xmlMinShare = (poolNode \ MINIMUM_SHARES_PROPERTY).text + if (xmlMinShare != "") { + minShare = xmlMinShare.toInt + } + + val xmlWeight = (poolNode \ WEIGHT_PROPERTY).text + if (xmlWeight != "") { + weight = xmlWeight.toInt + } + + val pool = new Pool(poolName, schedulingMode, minShare, weight) + rootPool.addSchedulable(pool) + logInfo("Created pool %s, schedulingMode: %s, minShare: %d, weight: %d".format( + poolName, schedulingMode, minShare, weight)) + } + } + override def addTaskSetManager(manager: Schedulable, properties: Properties) { var poolName = DEFAULT_POOL_NAME var parentPool = rootPool.getSchedulableByName(poolName) From 85024acd2ed9d6b2a03422214865ebb48b2094b8 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 13 Sep 2013 09:15:31 +0800 Subject: [PATCH 17/34] Remove infix style and others --- .../scheduler/cluster/SchedulableBuilder.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala index f25924befa..65b844a80e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala @@ -17,14 +17,12 @@ package org.apache.spark.scheduler.cluster -import java.io.{File, FileInputStream, FileOutputStream, FileNotFoundException, InputStream} +import java.io.{FileInputStream, InputStream} import java.util.Properties -import scala.xml.XML - import org.apache.spark.Logging -import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import scala.xml.XML /** * An interface to build Schedulable tree @@ -67,17 +65,17 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) override def buildPools() { var is: Option[InputStream] = None try { - is = Option { - schedulerAllocFile map { f => + is = Option{ + schedulerAllocFile.map{ f => new FileInputStream(f) - } getOrElse { + }.getOrElse{ getClass.getClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) } } - is foreach { i => buildFairSchedulerPool(i) } + is.foreach{ i => buildFairSchedulerPool(i) } } finally { - is.foreach(_.close) + is.foreach(_.close()) } // finally create "default" pool @@ -108,7 +106,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) try { schedulingMode = SchedulingMode.withName(xmlSchedulingMode) } catch { - case e: Exception => logInfo("Error xml schedulingMode, using default schedulingMode") + case e: Exception => logWarning("Error xml schedulingMode, using default schedulingMode") } } From 77e9da1f34a0b9e556d7c0bbd4aeaa5c635b881d Mon Sep 17 00:00:00 2001 From: jerryshao Date: Sun, 22 Sep 2013 16:50:08 +0800 Subject: [PATCH 18/34] Change Exception to NoSuchElementException and minor style fix --- .../scheduler/cluster/SchedulableBuilder.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala index 65b844a80e..114617c51a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster import java.io.{FileInputStream, InputStream} -import java.util.Properties +import java.util.{NoSuchElementException, Properties} import org.apache.spark.Logging @@ -65,15 +65,15 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) override def buildPools() { var is: Option[InputStream] = None try { - is = Option{ - schedulerAllocFile.map{ f => + is = Option { + schedulerAllocFile.map { f => new FileInputStream(f) - }.getOrElse{ + }.getOrElse { getClass.getClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) } } - is.foreach{ i => buildFairSchedulerPool(i) } + is.foreach { i => buildFairSchedulerPool(i) } } finally { is.foreach(_.close()) } @@ -106,7 +106,8 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) try { schedulingMode = SchedulingMode.withName(xmlSchedulingMode) } catch { - case e: Exception => logWarning("Error xml schedulingMode, using default schedulingMode") + case e: NoSuchElementException => + logWarning("Error xml schedulingMode, using default schedulingMode") } } From 7fe0b0ff568e8cf43efaabf46ac760cc8b91cf79 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sun, 22 Sep 2013 19:44:51 -0700 Subject: [PATCH 19/34] Switch indent from 2 to 4 spaces --- .../main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 264c4bc3de..b932c7c7cd 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -73,8 +73,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitionsWithIndex[R: ClassManifest](f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = - new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), - preservesPartitioning)) + new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), + preservesPartitioning)) /** * Return a new RDD by applying a function to all elements of this RDD. From c75eb14fe52f6789430983471974e5ddf73aacbf Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Sun, 12 May 2013 15:30:02 -0700 Subject: [PATCH 20/34] Send Task results through the block manager when larger than Akka frame size. This change requires adding an extra failure mode: tasks can complete successfully, but the result gets lost or flushed from the block manager before it's been fetched. --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/TaskEndReason.scala | 8 +- .../org/apache/spark/executor/Executor.scala | 26 ++- .../apache/spark/scheduler/DAGScheduler.scala | 5 +- .../apache/spark/scheduler/TaskResult.scala | 14 +- .../scheduler/cluster/ClusterScheduler.scala | 56 ++++-- .../cluster/ClusterTaskSetManager.scala | 171 ++++++++---------- .../apache/spark/scheduler/cluster/Pool.scala | 6 +- .../spark/scheduler/cluster/Schedulable.scala | 4 +- .../cluster/TaskResultResolver.scala | 125 +++++++++++++ .../scheduler/cluster/TaskSetManager.scala | 2 - .../scheduler/local/LocalScheduler.scala | 5 +- .../scheduler/local/LocalTaskSetManager.scala | 22 ++- .../apache/spark/storage/BlockManager.scala | 27 ++- .../org/apache/spark/DistributedSuite.scala | 13 -- .../scheduler/TaskResultResolverSuite.scala | 106 +++++++++++ .../cluster/ClusterSchedulerSuite.scala | 15 +- .../cluster/ClusterTaskSetManagerSuite.scala | 12 +- 18 files changed, 453 insertions(+), 166 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultResolver.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/TaskResultResolverSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 72540c712a..d9be6f71f2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -145,7 +145,7 @@ class SparkContext( } // Create and start the scheduler - private var taskScheduler: TaskScheduler = { + private[spark] var taskScheduler: TaskScheduler = { // Regular expression used for local[N] master format val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 03bf268863..8466c2a004 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -46,6 +46,10 @@ private[spark] case class ExceptionFailure( metrics: Option[TaskMetrics]) extends TaskEndReason -private[spark] case class OtherFailure(message: String) extends TaskEndReason +/** + * The task finished successfully, but the result was lost from the executor's block manager before + * it was fetched. + */ +private[spark] case object TaskResultLost extends TaskEndReason -private[spark] case class TaskResultTooBigFailure() extends TaskEndReason +private[spark] case class OtherFailure(message: String) extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index ceae3b8289..acdb8d0343 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import java.io.{File} +import java.io.File import java.lang.management.ManagementFactory import java.nio.ByteBuffer import java.util.concurrent._ @@ -27,11 +27,11 @@ import scala.collection.mutable.HashMap import org.apache.spark.scheduler._ import org.apache.spark._ +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils - /** - * The Mesos executor for Spark. + * Spark executor used with Mesos and the standalone scheduler. */ private[spark] class Executor( executorId: String, @@ -167,12 +167,20 @@ private[spark] class Executor( // we need to serialize the task metrics first. If TaskMetrics had a custom serialized format, we could // just change the relevants bytes in the byte buffer val accumUpdates = Accumulators.values - val result = new TaskResult(value, accumUpdates, task.metrics.getOrElse(null)) - val serializedResult = ser.serialize(result) - logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit) - if (serializedResult.limit >= (akkaFrameSize - 1024)) { - context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(TaskResultTooBigFailure())) - return + val directResult = new DirectTaskResult(value, accumUpdates, task.metrics.getOrElse(null)) + val serializedDirectResult = ser.serialize(directResult) + logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit) + val serializedResult = { + if (serializedDirectResult.limit >= akkaFrameSize - 1024) { + logInfo("Storing result for " + taskId + " in local BlockManager") + val blockId = "taskresult_" + taskId + env.blockManager.putBytes( + blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER) + ser.serialize(new IndirectTaskResult[Any](blockId)) + } else { + logInfo("Sending result for " + taskId + " directly to driver") + serializedDirectResult + } } context.statusUpdate(taskId, TaskState.FINISHED, serializedResult) logInfo("Finished task ID " + taskId) 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 3e3f04f087..db998e499a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -553,7 +553,7 @@ class DAGScheduler( SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) } catch { case e: NotSerializableException => - abortStage(stage, e.toString) + abortStage(stage, "Task not serializable: " + e.toString) running -= stage return } @@ -705,6 +705,9 @@ class DAGScheduler( case ExceptionFailure(className, description, stackTrace, metrics) => // Do nothing here, left up to the TaskScheduler to decide how to handle user failures + case TaskResultLost => + // Do nothing here; the TaskScheduler handles these failures and resubmits the task. + case other => // Unrecognized failure - abort all jobs depending on this stage abortStage(stageIdToStage(task.stageId), task + " failed: " + other) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 5c7e5bb977..25a61b3115 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -29,9 +29,17 @@ import org.apache.spark.util.Utils // TODO: Use of distributed cache to return result is a hack to get around // what seems to be a bug with messages over 60KB in libprocess; fix it private[spark] -class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) - extends Externalizable -{ +sealed abstract class TaskResult[T] + +/** A reference to a DirectTaskResult that has been stored in the worker's BlockManager. */ +private[spark] +case class IndirectTaskResult[T](val blockId: String) extends TaskResult[T] with Serializable + +/** A TaskResult that contains the task's return value and accumulator updates. */ +private[spark] +class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) + extends TaskResult[T] with Externalizable { + def this() = this(null.asInstanceOf[T], null, null) override def writeExternal(out: ObjectOutput) { 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 919acce828..db7c6001f1 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 @@ -18,6 +18,9 @@ package org.apache.spark.scheduler.cluster import java.lang.{Boolean => JBoolean} +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicLong +import java.util.{TimerTask, Timer} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -27,9 +30,7 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode -import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicLong -import java.util.{TimerTask, Timer} + /** * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call @@ -55,7 +56,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Threshold above which we warn user initial TaskSet may be starved val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong - val activeTaskSets = new HashMap[String, TaskSetManager] + // ClusterTaskSetManagers are not thread safe, so any access to one should be synchronized + // on this class. + val activeTaskSets = new HashMap[String, ClusterTaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskIdToExecutorId = new HashMap[Long, String] @@ -65,7 +68,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) @volatile private var hasLaunchedTask = false private val starvationTimer = new Timer(true) - // Incrementing Mesos task IDs + // Incrementing task IDs val nextTaskId = new AtomicLong(0) // Which executor IDs we have executors on @@ -96,6 +99,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val schedulingMode: SchedulingMode = SchedulingMode.withName( System.getProperty("spark.scheduler.mode", "FIFO")) + // This is a var so that we can reset it for testing purposes. + private[spark] var taskResultResolver = new TaskResultResolver(sc.env, this) + override def setListener(listener: TaskSchedulerListener) { this.listener = listener } @@ -234,7 +240,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - var taskSetToUpdate: Option[TaskSetManager] = None var failedExecutor: Option[String] = None var taskFailed = false synchronized { @@ -249,9 +254,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } taskIdToTaskSetId.get(tid) match { case Some(taskSetId) => - if (activeTaskSets.contains(taskSetId)) { - taskSetToUpdate = Some(activeTaskSets(taskSetId)) - } if (TaskState.isFinished(state)) { taskIdToTaskSetId.remove(tid) if (taskSetTaskIds.contains(taskSetId)) { @@ -262,6 +264,15 @@ private[spark] class ClusterScheduler(val sc: SparkContext) if (state == TaskState.FAILED) { taskFailed = true } + activeTaskSets.get(taskSetId).foreach { taskSet => + if (state == TaskState.FINISHED) { + taskSet.removeRunningTask(tid) + taskResultResolver.enqueueSuccessfulTask(taskSet, tid, serializedData) + } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { + taskSet.removeRunningTask(tid) + taskResultResolver.enqueueFailedTask(taskSet, tid, state, serializedData) + } + } case None => logInfo("Ignoring update from TID " + tid + " because its task set is gone") } @@ -269,10 +280,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) case e: Exception => logError("Exception in statusUpdate", e) } } - // Update the task set and DAGScheduler without holding a lock on this, since that can deadlock - if (taskSetToUpdate != None) { - taskSetToUpdate.get.statusUpdate(tid, state, serializedData) - } + // Update the DAGScheduler without holding a lock on this, since that can deadlock if (failedExecutor != None) { listener.executorLost(failedExecutor.get) backend.reviveOffers() @@ -283,6 +291,25 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } + def handleSuccessfulTask( + taskSetManager: ClusterTaskSetManager, + tid: Long, + taskResult: DirectTaskResult[_]) = synchronized { + taskSetManager.handleSuccessfulTask(tid, taskResult) + } + + def handleFailedTask( + taskSetManager: ClusterTaskSetManager, + tid: Long, + taskState: TaskState, + reason: Option[TaskEndReason]) = synchronized { + taskSetManager.handleFailedTask(tid, taskState, reason) + if (taskState == TaskState.FINISHED) { + // The task finished successfully but the result was lost, so we should revive offers. + backend.reviveOffers() + } + } + def error(message: String) { synchronized { if (activeTaskSets.size > 0) { @@ -311,6 +338,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) if (jarServer != null) { jarServer.stop() } + if (taskResultResolver != null) { + taskResultResolver.stop() + } // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out. // TODO: Do something better ! 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 0ac3d7bcfd..25e6f0a3ac 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 @@ -25,15 +25,13 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import scala.math.max import scala.math.min +import scala.Some -import org.apache.spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState} -import org.apache.spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} +import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler._ -import scala.Some import org.apache.spark.FetchFailed import org.apache.spark.ExceptionFailure -import org.apache.spark.TaskResultTooBigFailure import org.apache.spark.util.{SystemClock, Clock} @@ -71,18 +69,20 @@ private[spark] class ClusterTaskSetManager( val tasks = taskSet.tasks val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) - val finished = new Array[Boolean](numTasks) + val successful = new Array[Boolean](numTasks) val numFailures = new Array[Int](numTasks) val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) - var tasksFinished = 0 + var tasksSuccessful = 0 var weight = 1 var minShare = 0 - var runningTasks = 0 var priority = taskSet.priority var stageId = taskSet.stageId var name = "TaskSet_"+taskSet.stageId.toString - var parent: Schedulable = null + var parent: Pool = null + + var runningTasks = 0 + private val runningTasksSet = new HashSet[Long] // Set of pending tasks for each executor. These collections are actually // treated as stacks, in which new tasks are added to the end of the @@ -223,7 +223,7 @@ private[spark] class ClusterTaskSetManager( while (!list.isEmpty) { val index = list.last list.trimEnd(1) - if (copiesRunning(index) == 0 && !finished(index)) { + if (copiesRunning(index) == 0 && !successful(index)) { return Some(index) } } @@ -243,7 +243,7 @@ private[spark] class ClusterTaskSetManager( private def findSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value) : Option[(Int, TaskLocality.Value)] = { - speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set + speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set if (!speculatableTasks.isEmpty) { // Check for process-local or preference-less tasks; note that tasks can be process-local @@ -344,7 +344,7 @@ private[spark] class ClusterTaskSetManager( maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { - if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { + if (tasksSuccessful < numTasks && availableCpus >= CPUS_PER_TASK) { val curTime = clock.getTime() var allowedLocality = getAllowedLocalityLevel(curTime) @@ -375,7 +375,7 @@ private[spark] class ClusterTaskSetManager( val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) val timeTaken = clock.getTime() - startTime - increaseRunningTasks(1) + addRunningTask(taskId) logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) val taskName = "task %s:%d".format(taskSet.id, index) @@ -417,94 +417,63 @@ private[spark] class ClusterTaskSetManager( index } - /** Called by cluster scheduler when one of our tasks changes state */ - override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - SparkEnv.set(env) - state match { - case TaskState.FINISHED => - taskFinished(tid, state, serializedData) - case TaskState.LOST => - taskLost(tid, state, serializedData) - case TaskState.FAILED => - taskLost(tid, state, serializedData) - case TaskState.KILLED => - taskLost(tid, state, serializedData) - case _ => - } - } - - def taskStarted(task: Task[_], info: TaskInfo) { + private def taskStarted(task: Task[_], info: TaskInfo) { sched.listener.taskStarted(task, info) } - def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { + /** + * Marks the task as successful and notifies the listener that a task has ended. + */ + def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = { val info = taskInfos(tid) - if (info.failed) { - // We might get two task-lost messages for the same task in coarse-grained Mesos mode, - // or even from Mesos itself when acks get delayed. - return - } val index = info.index info.markSuccessful() - decreaseRunningTasks(1) - if (!finished(index)) { - tasksFinished += 1 + removeRunningTask(tid) + if (!successful(index)) { logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format( - tid, info.duration, info.host, tasksFinished, numTasks)) - // Deserialize task result and pass it to the scheduler - try { - val result = ser.deserialize[TaskResult[_]](serializedData) - result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded( - tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) - } catch { - case cnf: ClassNotFoundException => - val loader = Thread.currentThread().getContextClassLoader - throw new SparkException("ClassNotFound with classloader: " + loader, cnf) - case ex => throw ex - } - // Mark finished and stop if we've finished all the tasks - finished(index) = true - if (tasksFinished == numTasks) { + tid, info.duration, info.host, tasksSuccessful, numTasks)) + sched.listener.taskEnded( + tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + + // Mark successful and stop if all the tasks have succeeded. + tasksSuccessful += 1 + successful(index) = true + if (tasksSuccessful == numTasks) { sched.taskSetFinished(this) } } else { - logInfo("Ignoring task-finished event for TID " + tid + - " because task " + index + " is already finished") + logInfo("Ignorning task-finished event for TID " + tid + " because task " + + index + " has already completed successfully") } } - def taskLost(tid: Long, state: TaskState, serializedData: ByteBuffer) { + /** + * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the listener. + */ + def handleFailedTask(tid: Long, state: TaskState, reason: Option[TaskEndReason]) { val info = taskInfos(tid) if (info.failed) { - // We might get two task-lost messages for the same task in coarse-grained Mesos mode, - // or even from Mesos itself when acks get delayed. return } + removeRunningTask(tid) val index = info.index info.markFailed() - decreaseRunningTasks(1) - if (!finished(index)) { + // Count failed attempts only on FAILED and LOST state (not on KILLED) + var countFailedTaskAttempt = (state == TaskState.FAILED || state == TaskState.LOST) + if (!successful(index)) { logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) copiesRunning(index) -= 1 // Check if the problem is a map output fetch failure. In that case, this // task will never succeed on any node, so tell the scheduler about it. - if (serializedData != null && serializedData.limit() > 0) { - val reason = ser.deserialize[TaskEndReason](serializedData, getClass.getClassLoader) - reason match { + reason.foreach { + _ match { case fetchFailed: FetchFailed => logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null) - finished(index) = true - tasksFinished += 1 + successful(index) = true + tasksSuccessful += 1 sched.taskSetFinished(this) - decreaseRunningTasks(runningTasks) - return - - case taskResultTooBig: TaskResultTooBigFailure => - logInfo("Loss was due to task %s result exceeding Akka frame size; aborting job".format( - tid)) - abort("Task %s result exceeded Akka frame size".format(tid)) + removeAllRunningTasks() return case ef: ExceptionFailure => @@ -534,13 +503,17 @@ private[spark] class ClusterTaskSetManager( logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) } + case TaskResultLost => + logInfo("Lost result for TID %s on host %s".format(tid, info.host)) + countFailedTaskAttempt = true + sched.listener.taskEnded(tasks(index), TaskResultLost, null, null, info, null) + case _ => {} } } // On non-fetch failures, re-enqueue the task as pending for a max number of retries addPendingTask(index) - // Count failed attempts only on FAILED and LOST state (not on KILLED) - if (state == TaskState.FAILED || state == TaskState.LOST) { + if (countFailedTaskAttempt) { numFailures(index) += 1 if (numFailures(index) > MAX_TASK_FAILURES) { logError("Task %s:%d failed more than %d times; aborting job".format( @@ -564,22 +537,36 @@ private[spark] class ClusterTaskSetManager( causeOfFailure = message // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.listener.taskSetFailed(taskSet, message) - decreaseRunningTasks(runningTasks) + removeAllRunningTasks() sched.taskSetFinished(this) } - override def increaseRunningTasks(taskNum: Int) { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) + /** If the given task ID is not in the set of running tasks, adds it. + * + * Used to keep track of the number of running tasks, for enforcing scheduling policies. + */ + def addRunningTask(tid: Long) { + if (runningTasksSet.add(tid) && parent != null) { + parent.increaseRunningTasks(1) } + runningTasks = runningTasksSet.size } - override def decreaseRunningTasks(taskNum: Int) { - runningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) + /** If the given task ID is in the set of running tasks, removes it. */ + def removeRunningTask(tid: Long) { + if (runningTasksSet.remove(tid) && parent != null) { + parent.decreaseRunningTasks(1) } + runningTasks = runningTasksSet.size + } + + private def removeAllRunningTasks() { + val numRunningTasks = runningTasksSet.size + runningTasksSet.clear() + if (parent != null) { + parent.decreaseRunningTasks(numRunningTasks) + } + runningTasks = 0 } override def getSchedulableByName(name: String): Schedulable = { @@ -615,10 +602,10 @@ private[spark] class ClusterTaskSetManager( if (tasks(0).isInstanceOf[ShuffleMapTask]) { for ((tid, info) <- taskInfos if info.executorId == execId) { val index = taskInfos(tid).index - if (finished(index)) { - finished(index) = false + if (successful(index)) { + successful(index) = false copiesRunning(index) -= 1 - tasksFinished -= 1 + tasksSuccessful -= 1 addPendingTask(index) // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our // stage finishes when a total of tasks.size tasks finish. @@ -628,7 +615,7 @@ private[spark] class ClusterTaskSetManager( } // Also re-enqueue any tasks that were running on the node for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { - taskLost(tid, TaskState.KILLED, null) + handleFailedTask(tid, TaskState.KILLED, None) } } @@ -641,13 +628,13 @@ private[spark] class ClusterTaskSetManager( */ override def checkSpeculatableTasks(): Boolean = { // Can't speculate if we only have one task, or if all tasks have finished. - if (numTasks == 1 || tasksFinished == numTasks) { + if (numTasks == 1 || tasksSuccessful == numTasks) { return false } var foundTasks = false val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) - if (tasksFinished >= minFinishedForSpeculation) { + if (tasksSuccessful >= minFinishedForSpeculation) { val time = clock.getTime() val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray Arrays.sort(durations) @@ -658,7 +645,7 @@ private[spark] class ClusterTaskSetManager( logDebug("Task length threshold for speculation: " + threshold) for ((tid, info) <- taskInfos) { val index = info.index - if (!finished(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && + if (!successful(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && !speculatableTasks.contains(index)) { logInfo( "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format( @@ -672,7 +659,7 @@ private[spark] class ClusterTaskSetManager( } override def hasPendingTasks(): Boolean = { - numTasks > 0 && tasksFinished < numTasks + numTasks > 0 && tasksSuccessful < numTasks } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala index 35b32600da..199a0521ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala @@ -45,7 +45,7 @@ private[spark] class Pool( var priority = 0 var stageId = 0 var name = poolName - var parent:Schedulable = null + var parent: Pool = null var taskSetSchedulingAlgorithm: SchedulingAlgorithm = { schedulingMode match { @@ -101,14 +101,14 @@ private[spark] class Pool( return sortedTaskSetQueue } - override def increaseRunningTasks(taskNum: Int) { + def increaseRunningTasks(taskNum: Int) { runningTasks += taskNum if (parent != null) { parent.increaseRunningTasks(taskNum) } } - override def decreaseRunningTasks(taskNum: Int) { + def decreaseRunningTasks(taskNum: Int) { runningTasks -= taskNum if (parent != null) { parent.decreaseRunningTasks(taskNum) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala index f4726450ec..171549fbd9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer * there are two type of Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { - var parent: Schedulable + var parent: Pool // child queues def schedulableQueue: ArrayBuffer[Schedulable] def schedulingMode: SchedulingMode @@ -36,8 +36,6 @@ private[spark] trait Schedulable { def stageId: Int def name: String - def increaseRunningTasks(taskNum: Int): Unit - def decreaseRunningTasks(taskNum: Int): Unit def addSchedulable(schedulable: Schedulable): Unit def removeSchedulable(schedulable: Schedulable): Unit def getSchedulableByName(name: String): Schedulable diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultResolver.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultResolver.scala new file mode 100644 index 0000000000..812a9cf695 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultResolver.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.nio.ByteBuffer +import java.util.concurrent.{LinkedBlockingDeque, ThreadFactory, ThreadPoolExecutor, TimeUnit} + +import org.apache.spark._ +import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult} +import org.apache.spark.serializer.SerializerInstance + +/** + * Runs a thread pool that deserializes and remotely fetches (if neceessary) task results. + */ +private[spark] class TaskResultResolver(sparkEnv: SparkEnv, scheduler: ClusterScheduler) + extends Logging { + private val MIN_THREADS = 20 + private val MAX_THREADS = 60 + private val KEEP_ALIVE_SECONDS = 60 + private val getTaskResultExecutor = new ThreadPoolExecutor( + MIN_THREADS, + MAX_THREADS, + KEEP_ALIVE_SECONDS, + TimeUnit.SECONDS, + new LinkedBlockingDeque[Runnable], + new ResultResolverThreadFactory) + + class ResultResolverThreadFactory extends ThreadFactory { + private var counter = 0 + private var PREFIX = "Result resolver thread" + + override def newThread(r: Runnable): Thread = { + val thread = new Thread(r, "%s-%s".format(PREFIX, counter)) + counter += 1 + thread.setDaemon(true) + return thread + } + } + + protected val serializer = new ThreadLocal[SerializerInstance] { + override def initialValue(): SerializerInstance = { + return sparkEnv.closureSerializer.newInstance() + } + } + + def enqueueSuccessfulTask( + taskSetManager: ClusterTaskSetManager, tid: Long, serializedData: ByteBuffer) { + getTaskResultExecutor.execute(new Runnable { + override def run() { + try { + val result = serializer.get().deserialize[TaskResult[_]](serializedData) match { + case directResult: DirectTaskResult[_] => directResult + case IndirectTaskResult(blockId) => + logDebug("Fetching indirect task result for TID %s".format(tid)) + val serializedTaskResult = sparkEnv.blockManager.getRemoteBytes(blockId) + if (!serializedTaskResult.isDefined) { + /* We won't be able to get the task result if the machine that ran the task failed + * between when the task ended and when we tried to fetch the result, or if the + * block manager had to flush the result. */ + scheduler.handleFailedTask( + taskSetManager, tid, TaskState.FINISHED, Some(TaskResultLost)) + return + } + val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( + serializedTaskResult.get) + sparkEnv.blockManager.master.removeBlock(blockId) + deserializedResult + } + result.metrics.resultSize = serializedData.limit() + scheduler.handleSuccessfulTask(taskSetManager, tid, result) + } catch { + case cnf: ClassNotFoundException => + val loader = Thread.currentThread.getContextClassLoader + taskSetManager.abort("ClassNotFound with classloader: " + loader) + case ex => + taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) + } + } + }) + } + + def enqueueFailedTask(taskSetManager: ClusterTaskSetManager, tid: Long, taskState: TaskState, + serializedData: ByteBuffer) { + var reason: Option[TaskEndReason] = None + getTaskResultExecutor.execute(new Runnable { + override def run() { + try { + if (serializedData != null && serializedData.limit() > 0) { + reason = Some(serializer.get().deserialize[TaskEndReason]( + serializedData, getClass.getClassLoader)) + } + } catch { + case cnd: ClassNotFoundException => + // Log an error but keep going here -- the task failed, so not catastropic if we can't + // deserialize the reason. + val loader = Thread.currentThread.getContextClassLoader + logError( + "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) + case ex => {} + } + scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) + } + }) + } + + def stop() { + getTaskResultExecutor.shutdownNow() + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala index 648a3ef922..a0f3758a24 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala @@ -45,7 +45,5 @@ private[spark] trait TaskSetManager extends Schedulable { maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) - def error(message: String) } 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 8cb4d1396f..bcf9e1baf2 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 @@ -92,7 +92,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var rootPool: Pool = null val schedulingMode: SchedulingMode = SchedulingMode.withName( System.getProperty("spark.scheduler.mode", "FIFO")) - val activeTaskSets = new HashMap[String, TaskSetManager] + val activeTaskSets = new HashMap[String, LocalTaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -211,7 +211,8 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: deserializedTask.metrics.get.executorRunTime = serviceTime.toInt deserializedTask.metrics.get.jvmGCTime = getTotalGCTime - startGCTime deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt - val taskResult = new TaskResult(result, accumUpdates, deserializedTask.metrics.getOrElse(null)) + val taskResult = new DirectTaskResult( + result, accumUpdates, deserializedTask.metrics.getOrElse(null)) val serializedResult = ser.serialize(taskResult) localActor ! LocalStatusUpdate(taskId, TaskState.FINISHED, serializedResult) } catch { diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala index e52cb998bd..de0fd5a528 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala @@ -21,16 +21,17 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState} +import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, SparkException, Success, TaskState} import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler.{Task, TaskResult, TaskSet} -import org.apache.spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager} +import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskResult, TaskSet} +import org.apache.spark.scheduler.cluster.{Pool, Schedulable, TaskDescription, TaskInfo} +import org.apache.spark.scheduler.cluster.{TaskLocality, TaskSetManager} private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { - var parent: Schedulable = null + var parent: Pool = null var weight: Int = 1 var minShare: Int = 0 var runningTasks: Int = 0 @@ -49,14 +50,14 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val numFailures = new Array[Int](numTasks) val MAX_TASK_FAILURES = sched.maxFailures - override def increaseRunningTasks(taskNum: Int): Unit = { + def increaseRunningTasks(taskNum: Int): Unit = { runningTasks += taskNum if (parent != null) { parent.increaseRunningTasks(taskNum) } } - override def decreaseRunningTasks(taskNum: Int): Unit = { + def decreaseRunningTasks(taskNum: Int): Unit = { runningTasks -= taskNum if (parent != null) { parent.decreaseRunningTasks(taskNum) @@ -132,7 +133,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { SparkEnv.set(env) state match { case TaskState.FINISHED => @@ -152,7 +153,12 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val index = info.index val task = taskSet.tasks(index) info.markSuccessful() - val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) + val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) match { + case directResult: DirectTaskResult[_] => directResult + case IndirectTaskResult(blockId) => { + throw new SparkException("Expect only DirectTaskResults when using LocalScheduler") + } + } result.metrics.resultSize = serializedData.limit() sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics) numFinished += 1 diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 60fdc5f2ee..495a72db69 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -484,7 +484,7 @@ private[spark] class BlockManager( for (loc <- locations) { logDebug("Getting remote block " + blockId + " from " + loc) val data = BlockManagerWorker.syncGetBlock( - GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) + GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) if (data != null) { return Some(dataDeserialize(blockId, data)) } @@ -494,6 +494,31 @@ private[spark] class BlockManager( return None } + /** + * Get block from remote block managers as serialized bytes. + */ + def getRemoteBytes(blockId: String): Option[ByteBuffer] = { + // TODO: As with getLocalBytes, this is very similar to getRemote and perhaps should be + // refactored. + if (blockId == null) { + throw new IllegalArgumentException("Block Id is null") + } + logDebug("Getting remote block " + blockId + " as bytes") + + val locations = master.getLocations(blockId) + for (loc <- locations) { + logDebug("Getting remote block " + blockId + " from " + loc) + val data = BlockManagerWorker.syncGetBlock( + GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) + if (data != null) { + return Some(data) + } + logDebug("The value of block " + blockId + " is null") + } + logDebug("Block " + blockId + " not found") + return None + } + /** * Get a block from the block manager (either local or remote). */ diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 7a856d4081..cd2bf9a8ff 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -319,19 +319,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } } - - test("job should fail if TaskResult exceeds Akka frame size") { - // We must use local-cluster mode since results are returned differently - // when running under LocalScheduler: - sc = new SparkContext("local-cluster[1,1,512]", "test") - val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt - val rdd = sc.parallelize(Seq(1)).map{x => new Array[Byte](akkaFrameSize)} - val exception = intercept[SparkException] { - rdd.reduce((x, y) => x) - } - exception.getMessage should endWith("result exceeded Akka frame size") - } } object DistributedSuite { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultResolverSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultResolverSuite.scala new file mode 100644 index 0000000000..ff058c13ab --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultResolverSuite.scala @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 + +import java.nio.ByteBuffer + +import org.scalatest.BeforeAndAfter +import org.scalatest.FunSuite + +import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} +import org.apache.spark.scheduler.cluster.{ClusterScheduler, ClusterTaskSetManager, TaskResultResolver} + +/** + * Removes the TaskResult from the BlockManager before delegating to a normal TaskResultResolver. + * + * Used to test the case where a BlockManager evicts the task result (or dies) before the + * TaskResult is retrieved. + */ +class ResultDeletingTaskResultResolver(sparkEnv: SparkEnv, scheduler: ClusterScheduler) + extends TaskResultResolver(sparkEnv, scheduler) { + var removedResult = false + + override def enqueueSuccessfulTask( + taskSetManager: ClusterTaskSetManager, tid: Long, serializedData: ByteBuffer) { + if (!removedResult) { + // Only remove the result once, since we'd like to test the case where the task eventually + // succeeds. + serializer.get().deserialize[TaskResult[_]](serializedData) match { + case IndirectTaskResult(blockId) => + sparkEnv.blockManager.master.removeBlock(blockId) + case directResult: DirectTaskResult[_] => + taskSetManager.abort("Expect only indirect results") + } + serializedData.rewind() + removedResult = true + } + super.enqueueSuccessfulTask(taskSetManager, tid, serializedData) + } +} + +/** + * Tests related to handling task results (both direct and indirect). + */ +class TaskResultResolverSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { + + before { + // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small + // as we can make it) so the tests don't take too long. + System.setProperty("spark.akka.frameSize", "1") + // Use local-cluster mode because results are returned differently when running with the + // LocalScheduler. + sc = new SparkContext("local-cluster[1,1,512]", "test") + } + + test("handling results smaller than Akka frame size") { + val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) + assert(result === 2) + } + + test("handling results larger than Akka frame size") { + val akkaFrameSize = + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) + assert(result === 1.to(akkaFrameSize).toArray) + + val RESULT_BLOCK_ID = "taskresult_0" + assert(sc.env.blockManager.master.getLocations(RESULT_BLOCK_ID).size === 0, + "Expect result to be removed from the block manager.") + } + + test("task retried if result missing from block manager") { + // If this test hangs, it's probably because no resource offers were made after the task + // failed. + val scheduler: ClusterScheduler = sc.taskScheduler match { + case clusterScheduler: ClusterScheduler => + clusterScheduler + case _ => + assert(false, "Expect local cluster to use ClusterScheduler") + throw new ClassCastException + } + scheduler.taskResultResolver = new ResultDeletingTaskResultResolver(sc.env, scheduler) + val akkaFrameSize = + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) + assert(result === 1.to(akkaFrameSize).toArray) + + // Make sure two tasks were run (one failed one, and a second retried one). + assert(scheduler.nextTaskId.get() === 2) + } +} + diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala index 1b50ce06b3..95d3553d91 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala @@ -43,16 +43,16 @@ class FakeTaskSetManager( stageId = initStageId name = "TaskSet_"+stageId override val numTasks = initNumTasks - tasksFinished = 0 + tasksSuccessful = 0 - override def increaseRunningTasks(taskNum: Int) { + def increaseRunningTasks(taskNum: Int) { runningTasks += taskNum if (parent != null) { parent.increaseRunningTasks(taskNum) } } - override def decreaseRunningTasks(taskNum: Int) { + def decreaseRunningTasks(taskNum: Int) { runningTasks -= taskNum if (parent != null) { parent.decreaseRunningTasks(taskNum) @@ -79,7 +79,7 @@ class FakeTaskSetManager( maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { - if (tasksFinished + runningTasks < numTasks) { + if (tasksSuccessful + runningTasks < numTasks) { increaseRunningTasks(1) return Some(new TaskDescription(0, execId, "task 0:0", 0, null)) } @@ -92,8 +92,8 @@ class FakeTaskSetManager( def taskFinished() { decreaseRunningTasks(1) - tasksFinished +=1 - if (tasksFinished == numTasks) { + tasksSuccessful +=1 + if (tasksSuccessful == numTasks) { parent.removeSchedulable(this) } } @@ -114,7 +114,8 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging val taskSetQueue = rootPool.getSortedTaskSetQueue() /* Just for Test*/ for (manager <- taskSetQueue) { - logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format(manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) + logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( + manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) } for (taskSet <- taskSetQueue) { taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match { diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index ff70a2cdf0..ef99651b80 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -101,7 +101,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo assert(manager.resourceOffer("exec1", "host1", 2, PROCESS_LOCAL) === None) // Tell it the task has finished - manager.statusUpdate(0, TaskState.FINISHED, createTaskResult(0)) + manager.handleSuccessfulTask(0, createTaskResult(0)) assert(sched.endedTasks(0) === Success) assert(sched.finishedManagers.contains(manager)) } @@ -125,14 +125,14 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None) // Finish the first two tasks - manager.statusUpdate(0, TaskState.FINISHED, createTaskResult(0)) - manager.statusUpdate(1, TaskState.FINISHED, createTaskResult(1)) + manager.handleSuccessfulTask(0, createTaskResult(0)) + manager.handleSuccessfulTask(1, createTaskResult(1)) assert(sched.endedTasks(0) === Success) assert(sched.endedTasks(1) === Success) assert(!sched.finishedManagers.contains(manager)) // Finish the last task - manager.statusUpdate(2, TaskState.FINISHED, createTaskResult(2)) + manager.handleSuccessfulTask(2, createTaskResult(2)) assert(sched.endedTasks(2) === Success) assert(sched.finishedManagers.contains(manager)) } @@ -267,7 +267,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo new TaskSet(tasks, 0, 0, 0, null) } - def createTaskResult(id: Int): ByteBuffer = { - ByteBuffer.wrap(Utils.serialize(new TaskResult[Int](id, mutable.Map.empty, new TaskMetrics))) + def createTaskResult(id: Int): DirectTaskResult[Int] = { + new DirectTaskResult[Int](id, mutable.Map.empty, new TaskMetrics) } } From a314b30733adec4407e36be0de19328325ceb519 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Mon, 23 Sep 2013 14:48:17 -0500 Subject: [PATCH 21/34] Fix spacing so that the java.io.tmpdir doesn't run on with SPARK_JAVA_OPTS --- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 5 +++-- .../scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) 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 844c707834..3362010106 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 @@ -224,8 +224,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Add Xmx for am memory JAVA_OPTS += "-Xmx" + amMemory + "m " - JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), - YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + 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. @@ -241,6 +241,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " } + if (env.isDefinedAt("SPARK_JAVA_OPTS")) { JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " } 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 6229167cb4..a60e8a3007 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 @@ -77,8 +77,9 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " } - JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), - YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + 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 From 0cef683553414ba880d90527cc5f37e119efc782 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 23 Sep 2013 19:39:42 -0700 Subject: [PATCH 22/34] Fix formatting :) --- .../scala/org/apache/spark/api/java/JavaRDDLike.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index b932c7c7cd..7a3568c5ef 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -71,10 +71,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithIndex[R: ClassManifest](f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]], - preservesPartitioning: Boolean = false): JavaRDD[R] = - new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), - preservesPartitioning)) + def mapPartitionsWithIndex[R: ClassManifest]( + f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]], + preservesPartitioning: Boolean = false): JavaRDD[R] = + new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), + preservesPartitioning)) /** * Return a new RDD by applying a function to all elements of this RDD. From 6079721fa17cb2eeb0a9896405c75baaff0e98d7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 24 Sep 2013 11:41:51 -0700 Subject: [PATCH 23/34] Update build version in master --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- project/SparkBuild.scala | 2 +- python/pyspark/shell.py | 2 +- repl-bin/pom.xml | 2 +- repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 14 files changed, 15 insertions(+), 15 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 808a829e19..d62332137a 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index 51173c32b2..c4ce006085 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 14cd520aaf..9c2d6046a9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index b061764b36..48ecb8d0c9 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.0-SNAPSHOT -SPARK_VERSION_SHORT: 0.8.0 +SPARK_VERSION: 0.9.0-incubating-SNAPSHOT +SPARK_VERSION_SHORT: 0.9.0-SNAPSHOT SCALA_VERSION: 2.9.3 MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net diff --git a/examples/pom.xml b/examples/pom.xml index e48f5b50ab..b9cc6f5e0a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 966caf6835..4ef4f0ae4e 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4aed1260f0..d74d45adf1 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT pom Spark Project Parent POM http://spark.incubator.apache.org/ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ed7671757b..a2e29591fc 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.0-SNAPSHOT", + version := "0.9.0-incubating-SNAPSHOT", scalaVersion := "2.9.3", scalacOptions := Seq("-unchecked", "-optimize", "-deprecation", "-target:" + SCALAC_JVM_VERSION), diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index dc205b306f..a475959090 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -35,7 +35,7 @@ print """Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /__ / .__/\_,_/_/ /_/\_\ version 0.8.0 + /__ / .__/\_,_/_/ /_/\_\ version 0.9.0-SNAPSHOT /_/ """ print "Using Python version %s (%s, %s)" % ( diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 3685561501..05aadc7bdf 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml 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 193ccb48ee..36f54a22cf 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.0 + /___/ .__/\_,_/_/ /_/\_\ version 0.9.0-SNAPSHOT /_/ """) import Properties._ diff --git a/streaming/pom.xml b/streaming/pom.xml index 7bea069b61..b260a72abb 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 77646a6816..29f0014128 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 21b650d1ea..427fcdf545 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml From d85fe41b2b380e2879cb18008dbeb344ed7d7c92 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 25 Sep 2013 12:18:32 -0700 Subject: [PATCH 24/34] Improved organization of scheduling packages. This commit does not change any code -- only file organization. There are two components of this change: (1) Moving files out of the cluster package, and down a level to the scheduling package. These files are all used by the local scheduler in addition to the cluster scheduler(s), so should not be in the cluster package. As a result of this change, none of the files in the local package reference files in the cluster package. (2) Moving the mesos package to within the cluster package. The mesos scheduling code is for a cluster, and represents a specific case of cluster scheduling (the Mesos-related classes often subclass cluster scheduling classes). Thus, the most logical place for it is within the cluster package. --- .../scala/org/apache/spark/SparkContext.scala | 4 ++-- .../apache/spark/scheduler/DAGScheduler.scala | 1 - .../spark/scheduler/DAGSchedulerEvent.scala | 1 - .../apache/spark/scheduler/JobLogger.scala | 1 - .../spark/scheduler/{cluster => }/Pool.scala | 4 ++-- .../scheduler/{cluster => }/Schedulable.scala | 4 ++-- .../{cluster => }/SchedulableBuilder.scala | 2 +- .../{cluster => }/SchedulingAlgorithm.scala | 2 +- .../{cluster => }/SchedulingMode.scala | 2 +- .../spark/scheduler/SparkListener.scala | 1 - .../apache/spark/scheduler/StageInfo.scala | 2 +- .../{cluster => }/TaskDescription.scala | 2 +- .../scheduler/{cluster => }/TaskInfo.scala | 2 +- .../{cluster => }/TaskLocality.scala | 2 +- .../spark/scheduler/TaskScheduler.scala | 5 +++-- .../scheduler/TaskSchedulerListener.scala | 1 - .../{cluster => }/TaskSetManager.scala | 3 +-- .../scheduler/cluster/ClusterScheduler.scala | 2 +- .../cluster/ClusterTaskSetManager.scala | 9 +++----- .../cluster/StandaloneClusterMessage.scala | 1 + .../cluster/StandaloneSchedulerBackend.scala | 1 + .../mesos/CoarseMesosSchedulerBackend.scala | 20 ++++++++--------- .../mesos/MesosSchedulerBackend.scala | 22 ++++++++++--------- .../scheduler/local/LocalScheduler.scala | 3 +-- .../scheduler/local/LocalTaskSetManager.scala | 4 ++-- .../apache/spark/ui/UIWorkloadGenerator.scala | 2 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 2 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 2 +- .../spark/ui/jobs/JobProgressListener.scala | 4 +--- .../apache/spark/ui/jobs/JobProgressUI.scala | 4 ++-- .../org/apache/spark/ui/jobs/PoolTable.scala | 3 +-- .../org/apache/spark/ui/jobs/StagePage.scala | 6 ++--- .../org/apache/spark/ui/jobs/StageTable.scala | 3 +-- .../spark/scheduler/DAGSchedulerSuite.scala | 6 ++--- 34 files changed, 62 insertions(+), 71 deletions(-) rename core/src/main/scala/org/apache/spark/scheduler/{cluster => }/Pool.scala (96%) rename core/src/main/scala/org/apache/spark/scheduler/{cluster => }/Schedulable.scala (93%) rename core/src/main/scala/org/apache/spark/scheduler/{cluster => }/SchedulableBuilder.scala (99%) rename core/src/main/scala/org/apache/spark/scheduler/{cluster => }/SchedulingAlgorithm.scala (98%) rename core/src/main/scala/org/apache/spark/scheduler/{cluster => }/SchedulingMode.scala (96%) rename core/src/main/scala/org/apache/spark/scheduler/{cluster => }/TaskDescription.scala (97%) rename core/src/main/scala/org/apache/spark/scheduler/{cluster => }/TaskInfo.scala (97%) rename core/src/main/scala/org/apache/spark/scheduler/{cluster => }/TaskLocality.scala (96%) rename core/src/main/scala/org/apache/spark/scheduler/{cluster => }/TaskSetManager.scala (95%) rename core/src/main/scala/org/apache/spark/scheduler/{ => cluster}/mesos/CoarseMesosSchedulerBackend.scala (98%) rename core/src/main/scala/org/apache/spark/scheduler/{ => cluster}/mesos/MesosSchedulerBackend.scala (97%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6bab1f31d0..912ce752fb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -56,9 +56,9 @@ import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, - ClusterScheduler, Schedulable, SchedulingMode} + ClusterScheduler} import org.apache.spark.scheduler.local.LocalScheduler -import org.apache.spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.storage.{StorageUtils, BlockManagerSource} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, Utils, MetadataCleaner, TimeStampedHashMap} 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 3e3f04f087..8a55df4af0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -28,7 +28,6 @@ import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.storage.{BlockManager, BlockManagerMaster} import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} 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 0d99670648..10ff1b4376 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import java.util.Properties -import org.apache.spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index c8b78bf00a..3628b1b078 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -30,7 +30,6 @@ import scala.io.Source import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.cluster.TaskInfo // Used to record runtime information for each job, including RDD graph // tasks' start/stop shuffle information and information from outside diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala rename to core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 35b32600da..c9a66b3a75 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.scheduler import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import org.apache.spark.Logging -import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** * An Schedulable entity that represent collection of Pools or TaskSetManagers diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala rename to core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index f4726450ec..857adaef5a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.scheduler -import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import scala.collection.mutable.ArrayBuffer /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala rename to core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 114617c51a..4e25086ec9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.scheduler import java.io.{FileInputStream, InputStream} import java.util.{NoSuchElementException, Properties} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala rename to core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala index cbeed4731a..3418640b8c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.scheduler /** * An interface for sort algorithm diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala rename to core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala index 34811389a0..0a786deb16 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.scheduler /** * "FAIR" and "FIFO" determines which policy is used 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 c3cf4b8907..62b521ad45 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.util.Properties -import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.{Logging, SparkContext, TaskEndReason} import org.apache.spark.executor.TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 72cb1c9ce8..b6f11969e5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,8 +17,8 @@ package org.apache.spark.scheduler -import org.apache.spark.scheduler.cluster.TaskInfo import scala.collection._ + import org.apache.spark.executor.TaskMetrics case class StageInfo( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index 309ac2f6c9..5190d234d4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.scheduler import java.nio.ByteBuffer import org.apache.spark.util.SerializableBuffer diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 9685fb1a67..7c2a422aff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.scheduler import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 5d4130e14a..47b0f387aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.scheduler private[spark] object TaskLocality diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 63be8ba3f5..7c2a9f03d7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -17,10 +17,11 @@ package org.apache.spark.scheduler -import org.apache.spark.scheduler.cluster.Pool -import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode + /** * Low-level task scheduler interface, implemented by both ClusterScheduler and LocalScheduler. + * Each TaskScheduler schedulers task for a single SparkContext. * These schedulers get sets of tasks submitted to them from the DAGScheduler for each stage, * and are responsible for sending the tasks to the cluster, running them, retrying if there * are failures, and mitigating stragglers. They return events to the DAGScheduler through diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala index 83be051c1a..593fa9fb93 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import org.apache.spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map import org.apache.spark.TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 648a3ef922..f192b0b7a4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.scheduler import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler.TaskSet /** * Tracks and schedules the tasks within a single TaskSet. This class keeps track of the status of diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 919acce828..a6dee604b7 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 @@ -26,7 +26,7 @@ import scala.collection.mutable.HashSet import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong import java.util.{TimerTask, Timer} 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 0ac3d7bcfd..411e49b021 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 @@ -25,15 +25,12 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import scala.math.max import scala.math.min +import scala.Some -import org.apache.spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState} -import org.apache.spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} +import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv, + SparkException, Success, TaskEndReason, TaskResultTooBigFailure, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler._ -import scala.Some -import org.apache.spark.FetchFailed -import org.apache.spark.ExceptionFailure -import org.apache.spark.TaskResultTooBigFailure import org.apache.spark.util.{SystemClock, Clock} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala index 9c36d221f6..c0b836bf1a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler.TaskDescription import org.apache.spark.util.{Utils, SerializableBuffer} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index b4ea0be415..f3aeea43d5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -29,6 +29,7 @@ import akka.util.Duration import akka.util.duration._ import org.apache.spark.{SparkException, Logging, TaskState} +import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 3dbe61d706..8f2eef9a53 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -15,22 +15,22 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos + +import java.io.File +import java.util.{ArrayList => JArrayList, List => JList} +import java.util.Collections + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.JavaConversions._ import com.google.protobuf.ByteString - import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{SparkException, Logging, SparkContext} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.collection.JavaConversions._ -import java.io.File -import org.apache.spark.scheduler.cluster._ -import java.util.{ArrayList => JArrayList, List => JList} -import java.util.Collections -import org.apache.spark.TaskState +import org.apache.spark.{SparkException, Logging, SparkContext, TaskState} +import org.apache.spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend} /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 541f86e338..50cbc2ca92 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -15,22 +15,24 @@ * limitations under the License. */ -package org.apache.spark.scheduler.mesos +package org.apache.spark.scheduler.cluster.mesos + +import java.io.File +import java.util.{ArrayList => JArrayList, List => JList} +import java.util.Collections + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.JavaConversions._ import com.google.protobuf.ByteString - import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{SparkException, Logging, SparkContext} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.collection.JavaConversions._ -import java.io.File -import org.apache.spark.scheduler.cluster._ -import java.util.{ArrayList => JArrayList, List => JList} -import java.util.Collections -import org.apache.spark.TaskState +import org.apache.spark.{Logging, SparkException, SparkContext, TaskState} +import org.apache.spark.scheduler.TaskDescription +import org.apache.spark.scheduler.cluster.{ClusterScheduler, ExecutorExited, ExecutorLossReason} +import org.apache.spark.scheduler.cluster.{SchedulerBackend, SlaveLost, WorkerOffer} import org.apache.spark.util.Utils /** 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 8cb4d1396f..e29438f4ed 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 @@ -31,8 +31,7 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.ExecutorURLClassLoader import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster._ -import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import akka.actor._ import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala index e52cb998bd..a2fda4c124 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala @@ -23,8 +23,8 @@ import scala.collection.mutable.HashMap import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState} import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler.{Task, TaskResult, TaskSet} -import org.apache.spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager} +import org.apache.spark.scheduler.{Schedulable, Task, TaskDescription, TaskInfo, TaskLocality, + TaskResult, TaskSet, TaskSetManager} private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 3ec9760ed0..453394dfda 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -21,7 +21,7 @@ import scala.util.Random import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.scheduler.cluster.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode /** 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 d1868dcf78..42e9be6e19 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 @@ -26,8 +26,8 @@ import org.eclipse.jetty.server.Handler import org.apache.spark.{ExceptionFailure, Logging, SparkContext} import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} +import org.apache.spark.scheduler.TaskInfo import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui.UIUtils diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 3b428effaf..b39c0e9769 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{NodeSeq, Node} -import org.apache.spark.scheduler.cluster.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils._ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 5d46f38a2a..eb3b4e8522 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -21,10 +21,8 @@ import scala.Seq import scala.collection.mutable.{ListBuffer, HashMap, HashSet} import org.apache.spark.{ExceptionFailure, SparkContext, Success} -import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.executor.TaskMetrics -import collection.mutable +import org.apache.spark.scheduler._ /** * Tracks task-level information to be displayed in the UI. diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 6aecef5120..e7eab374ad 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -32,8 +32,8 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.scheduler._ import collection.mutable -import org.apache.spark.scheduler.cluster.SchedulingMode -import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index b3d3666944..06810d8dbc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -21,8 +21,7 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import scala.xml.Node -import org.apache.spark.scheduler.Stage -import org.apache.spark.scheduler.cluster.Schedulable +import org.apache.spark.scheduler.{Schedulable, Stage} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ 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 a9969ab1c0..163a3746ea 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 @@ -23,12 +23,12 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.apache.spark.{ExceptionFailure} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ import org.apache.spark.util.{Utils, Distribution} -import org.apache.spark.{ExceptionFailure} -import org.apache.spark.scheduler.cluster.TaskInfo -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.TaskInfo /** Page showing statistics and task list for a given stage */ private[spark] class StagePage(parent: JobProgressUI) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 32776eaa25..07db8622da 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -22,8 +22,7 @@ import java.util.Date import scala.xml.Node import scala.collection.mutable.HashSet -import org.apache.spark.scheduler.cluster.{SchedulingMode, TaskInfo} -import org.apache.spark.scheduler.Stage +import org.apache.spark.scheduler.{SchedulingMode, Stage, TaskInfo} import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils 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 94f66c94c6..9ed591e494 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -32,9 +32,9 @@ import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency} import org.apache.spark.{FetchFailed, Success, TaskEndReason} import org.apache.spark.storage.{BlockManagerId, BlockManagerMaster} -import org.apache.spark.scheduler.cluster.Pool -import org.apache.spark.scheduler.cluster.SchedulingMode -import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.Pool +import org.apache.spark.scheduler.SchedulingMode +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler From e2ff59af728939b173cc12fa3368208a227fbaa2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 26 Sep 2013 10:26:50 -0700 Subject: [PATCH 25/34] Bug fix in master build --- repl/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/repl/pom.xml b/repl/pom.xml index 3123b37780..2826c0743c 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 0.8.0-SNAPSHOT + 0.9.0-incubating-SNAPSHOT ../pom.xml From 3f283278b00fc0a98a6c8cccd704bfc476f5d765 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 26 Sep 2013 13:58:10 -0700 Subject: [PATCH 26/34] Removed scala -optimize flag. --- pom.xml | 1 - project/SparkBuild.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index d74d45adf1..ad5051d38a 100644 --- a/pom.xml +++ b/pom.xml @@ -557,7 +557,6 @@ true -unchecked - -optimise -deprecation diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 99cdadb9e7..aef246d8a9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -81,7 +81,7 @@ object SparkBuild extends Build { organization := "org.apache.spark", version := "0.9.0-incubating-SNAPSHOT", scalaVersion := "2.9.3", - scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-optimize", "-deprecation", + scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, From 58b764b7c6b44bcacab2192bfd5200e3d689646e Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 30 Sep 2013 10:11:59 -0700 Subject: [PATCH 27/34] Addressed Matei's code review comments --- .../apache/spark/scheduler/TaskResult.scala | 5 +--- .../scheduler/cluster/ClusterScheduler.scala | 10 ++++---- .../cluster/ClusterTaskSetManager.scala | 6 +---- ...tResolver.scala => TaskResultGetter.scala} | 11 ++++---- .../cluster/ClusterTaskSetManagerSuite.scala | 17 +++++++++++++ .../TaskResultGetterSuite.scala} | 25 ++++++++++++------- 6 files changed, 45 insertions(+), 29 deletions(-) rename core/src/main/scala/org/apache/spark/scheduler/cluster/{TaskResultResolver.scala => TaskResultGetter.scala} (94%) rename core/src/test/scala/org/apache/spark/scheduler/{TaskResultResolverSuite.scala => cluster/TaskResultGetterSuite.scala} (84%) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 25a61b3115..db3954a9d3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -26,10 +26,7 @@ import java.nio.ByteBuffer import org.apache.spark.util.Utils // Task result. Also contains updates to accumulator variables. -// TODO: Use of distributed cache to return result is a hack to get around -// what seems to be a bug with messages over 60KB in libprocess; fix it -private[spark] -sealed abstract class TaskResult[T] +private[spark] sealed trait TaskResult[T] /** A reference to a DirectTaskResult that has been stored in the worker's BlockManager. */ private[spark] 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 db7c6001f1..299c0e1888 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 @@ -100,7 +100,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) System.getProperty("spark.scheduler.mode", "FIFO")) // This is a var so that we can reset it for testing purposes. - private[spark] var taskResultResolver = new TaskResultResolver(sc.env, this) + private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this) override def setListener(listener: TaskSchedulerListener) { this.listener = listener @@ -267,10 +267,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext) activeTaskSets.get(taskSetId).foreach { taskSet => if (state == TaskState.FINISHED) { taskSet.removeRunningTask(tid) - taskResultResolver.enqueueSuccessfulTask(taskSet, tid, serializedData) + taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData) } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) { taskSet.removeRunningTask(tid) - taskResultResolver.enqueueFailedTask(taskSet, tid, state, serializedData) + taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData) } } case None => @@ -338,8 +338,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) if (jarServer != null) { jarServer.stop() } - if (taskResultResolver != null) { - taskResultResolver.stop() + if (taskResultGetter != null) { + taskResultGetter.stop() } // sleeping for an arbitrary 5 seconds : to ensure that messages are sent out. 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 25e6f0a3ac..fc4da5df54 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 @@ -25,7 +25,6 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import scala.Some import org.apache.spark._ import org.apache.spark.TaskState.TaskState @@ -458,8 +457,6 @@ private[spark] class ClusterTaskSetManager( removeRunningTask(tid) val index = info.index info.markFailed() - // Count failed attempts only on FAILED and LOST state (not on KILLED) - var countFailedTaskAttempt = (state == TaskState.FAILED || state == TaskState.LOST) if (!successful(index)) { logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) copiesRunning(index) -= 1 @@ -505,7 +502,6 @@ private[spark] class ClusterTaskSetManager( case TaskResultLost => logInfo("Lost result for TID %s on host %s".format(tid, info.host)) - countFailedTaskAttempt = true sched.listener.taskEnded(tasks(index), TaskResultLost, null, null, info, null) case _ => {} @@ -513,7 +509,7 @@ private[spark] class ClusterTaskSetManager( } // On non-fetch failures, re-enqueue the task as pending for a max number of retries addPendingTask(index) - if (countFailedTaskAttempt) { + if (state != TaskState.KILLED) { numFailures(index) += 1 if (numFailures(index) > MAX_TASK_FAILURES) { logError("Task %s:%d failed more than %d times; aborting job".format( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultResolver.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala similarity index 94% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultResolver.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala index 812a9cf695..feec8ecfe4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultResolver.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala @@ -26,17 +26,16 @@ import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskRes import org.apache.spark.serializer.SerializerInstance /** - * Runs a thread pool that deserializes and remotely fetches (if neceessary) task results. + * Runs a thread pool that deserializes and remotely fetches (if necessary) task results. */ -private[spark] class TaskResultResolver(sparkEnv: SparkEnv, scheduler: ClusterScheduler) +private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) extends Logging { - private val MIN_THREADS = 20 - private val MAX_THREADS = 60 - private val KEEP_ALIVE_SECONDS = 60 + private val MIN_THREADS = System.getProperty("spark.resultGetter.minThreads", "4").toInt + private val MAX_THREADS = System.getProperty("spark.resultGetter.maxThreads", "4").toInt private val getTaskResultExecutor = new ThreadPoolExecutor( MIN_THREADS, MAX_THREADS, - KEEP_ALIVE_SECONDS, + 0L, TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable], new ResultResolverThreadFactory) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index ef99651b80..58cc1ef185 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -253,6 +253,23 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None) } + test("task result lost") { + sc = new SparkContext("local", "test") + val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) + val taskSet = createTaskSet(1) + val clock = new FakeClock + val manager = new ClusterTaskSetManager(sched, taskSet, clock) + + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + + // Tell it the task has finished but the result was lost. + manager.handleFailedTask(0, TaskState.FINISHED, Some(TaskResultLost)) + assert(sched.endedTasks(0) === TaskResultLost) + + // Re-offer the host -- now we should get task 0 again. + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + } + /** * Utility method to create a TaskSet, potentially setting a particular sequence of preferred * locations for each task (given as varargs) if this sequence is not empty. diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultResolverSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala similarity index 84% rename from core/src/test/scala/org/apache/spark/scheduler/TaskResultResolverSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala index ff058c13ab..2c201d15a7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler +package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer @@ -23,16 +23,16 @@ import org.scalatest.BeforeAndAfter import org.scalatest.FunSuite import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} -import org.apache.spark.scheduler.cluster.{ClusterScheduler, ClusterTaskSetManager, TaskResultResolver} +import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult} /** - * Removes the TaskResult from the BlockManager before delegating to a normal TaskResultResolver. + * Removes the TaskResult from the BlockManager before delegating to a normal TaskResultGetter. * * Used to test the case where a BlockManager evicts the task result (or dies) before the * TaskResult is retrieved. */ -class ResultDeletingTaskResultResolver(sparkEnv: SparkEnv, scheduler: ClusterScheduler) - extends TaskResultResolver(sparkEnv, scheduler) { +class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler) + extends TaskResultGetter(sparkEnv, scheduler) { var removedResult = false override def enqueueSuccessfulTask( @@ -44,7 +44,7 @@ class ResultDeletingTaskResultResolver(sparkEnv: SparkEnv, scheduler: ClusterSch case IndirectTaskResult(blockId) => sparkEnv.blockManager.master.removeBlock(blockId) case directResult: DirectTaskResult[_] => - taskSetManager.abort("Expect only indirect results") + taskSetManager.abort("Internal error: expect only indirect results") } serializedData.rewind() removedResult = true @@ -56,9 +56,11 @@ class ResultDeletingTaskResultResolver(sparkEnv: SparkEnv, scheduler: ClusterSch /** * Tests related to handling task results (both direct and indirect). */ -class TaskResultResolverSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { +class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { + + override def beforeAll() { + super.beforeAll() - before { // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small // as we can make it) so the tests don't take too long. System.setProperty("spark.akka.frameSize", "1") @@ -67,6 +69,11 @@ class TaskResultResolverSuite extends FunSuite with BeforeAndAfter with LocalSpa sc = new SparkContext("local-cluster[1,1,512]", "test") } + override def afterAll() { + super.afterAll() + System.clearProperty("spark.akka.frameSize") + } + test("handling results smaller than Akka frame size") { val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) assert(result === 2) @@ -93,7 +100,7 @@ class TaskResultResolverSuite extends FunSuite with BeforeAndAfter with LocalSpa assert(false, "Expect local cluster to use ClusterScheduler") throw new ClassCastException } - scheduler.taskResultResolver = new ResultDeletingTaskResultResolver(sc.env, scheduler) + scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) From dea4677c887a515e7b2a3ef52dd65e69b15c60c3 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 30 Sep 2013 16:44:45 -0700 Subject: [PATCH 28/34] Fixed compilation errors and broken test. --- .../cluster/ClusterTaskSetManager.scala | 2 +- .../scheduler/local/LocalTaskSetManager.scala | 4 ++-- .../spark/scheduler/DAGSchedulerSuite.scala | 2 -- .../cluster/TaskResultGetterSuite.scala | 16 ++++++++-------- 4 files changed, 11 insertions(+), 13 deletions(-) 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 c7225de870..194ab55102 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 @@ -28,7 +28,7 @@ import scala.math.min import scala.Some import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv, - SparkException, Success, TaskEndReason, TaskResultTooBigFailure, TaskState} + SparkException, Success, TaskEndReason, TaskResultLost, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler._ import org.apache.spark.util.{SystemClock, Clock} diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala index dc4cf555de..c2e2399ccb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala @@ -23,8 +23,8 @@ import scala.collection.mutable.HashMap import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, SparkException, Success, TaskState} import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler.{Schedulable, Task, TaskDescription, TaskInfo, TaskLocality, - TaskResult, TaskSet, TaskSetManager} +import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Pool, Schedulable, Task, + TaskDescription, TaskInfo, TaskLocality, TaskResult, TaskSet, TaskSetManager} private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) 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 9ed591e494..2f933246b0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -32,8 +32,6 @@ import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency} import org.apache.spark.{FetchFailed, Success, TaskEndReason} import org.apache.spark.storage.{BlockManagerId, BlockManagerMaster} -import org.apache.spark.scheduler.Pool -import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala index 2c201d15a7..119ba30090 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala @@ -19,8 +19,7 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer -import org.scalatest.BeforeAndAfter -import org.scalatest.FunSuite +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult} @@ -56,21 +55,22 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSched /** * Tests related to handling task results (both direct and indirect). */ -class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - - override def beforeAll() { - super.beforeAll() +class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll + with LocalSparkContext { + override def beforeAll { // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small // as we can make it) so the tests don't take too long. System.setProperty("spark.akka.frameSize", "1") + } + + before { // Use local-cluster mode because results are returned differently when running with the // LocalScheduler. sc = new SparkContext("local-cluster[1,1,512]", "test") } - override def afterAll() { - super.afterAll() + override def afterAll { System.clearProperty("spark.akka.frameSize") } From 0dcad2edcbcc1f3f12a339110e85c8b1a48af156 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 30 Sep 2013 23:26:15 -0700 Subject: [PATCH 29/34] Added additional unit test for repeated task failures --- .../cluster/ClusterTaskSetManagerSuite.scala | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index 58cc1ef185..80d0c5a5e9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -40,6 +40,7 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* val startedTasks = new ArrayBuffer[Long] val endedTasks = new mutable.HashMap[Long, TaskEndReason] val finishedManagers = new ArrayBuffer[TaskSetManager] + val taskSetsFailed = new ArrayBuffer[String] val executors = new mutable.HashMap[String, String] ++ liveExecutors @@ -63,7 +64,9 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* def executorLost(execId: String) {} - def taskSetFailed(taskSet: TaskSet, reason: String) {} + def taskSetFailed(taskSet: TaskSet, reason: String) { + taskSetsFailed += taskSet.id + } } def removeExecutor(execId: String): Unit = executors -= execId @@ -270,6 +273,30 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) } + test("repeated failures lead to task set abortion") { + sc = new SparkContext("local", "test") + val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) + val taskSet = createTaskSet(1) + val clock = new FakeClock + val manager = new ClusterTaskSetManager(sched, taskSet, clock) + + // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted + // after the last failure. + (0 until manager.MAX_TASK_FAILURES).foreach { index => + val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY) + assert(offerResult != None, + "Expect resource offer on iteration %s to return a task".format(index)) + assert(offerResult.get.index === 0) + manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, Some(TaskResultLost)) + if (index < manager.MAX_TASK_FAILURES) { + assert(!sched.taskSetsFailed.contains(taskSet.id)) + } else { + assert(sched.taskSetsFailed.contains(taskSet.id)) + } + } + } + + /** * Utility method to create a TaskSet, potentially setting a particular sequence of preferred * locations for each task (given as varargs) if this sequence is not empty. From c85f72058871afe46e416f5728be3b5c2f6ba46a Mon Sep 17 00:00:00 2001 From: CruncherBigData Date: Tue, 1 Oct 2013 09:05:03 -0700 Subject: [PATCH 30/34] Update README --- ec2/README | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/README b/ec2/README index 0add81312c..433da37b4c 100644 --- a/ec2/README +++ b/ec2/README @@ -1,4 +1,4 @@ This folder contains a script, spark-ec2, for launching Spark clusters on Amazon EC2. Usage instructions are available online at: -http://spark-project.org/docs/latest/ec2-scripts.html +http://spark.incubator.apache.org/docs/latest/ec2-scripts.html From 0d19f00e9e92b47053822b49ec6de502b5df6241 Mon Sep 17 00:00:00 2001 From: Du Li Date: Tue, 1 Oct 2013 15:42:06 -0700 Subject: [PATCH 31/34] fixed a bug of using wildcard in quotes --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index bffb19843c..32bbdb90a5 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -95,7 +95,7 @@ cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/jars/" # Copy other things mkdir "$DISTDIR"/conf -cp "$FWDIR/conf/*.template" "$DISTDIR"/conf +cp "$FWDIR"/conf/*.template "$DISTDIR"/conf cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp "$FWDIR/spark-class" "$DISTDIR" From 9fd6bba60d908c1c176e3bbd34add1853ecc1d8d Mon Sep 17 00:00:00 2001 From: Du Li Date: Tue, 1 Oct 2013 15:46:51 -0700 Subject: [PATCH 32/34] ask ivy/sbt to check local maven repo under ~/.m2 --- project/SparkBuild.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index aef246d8a9..cdec6168af 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -97,6 +97,9 @@ object SparkBuild extends Build { // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), + // also check the local Maven repository ~/.m2 + resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))), + // Shared between both core and streaming. resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), From bc3b20abdcceb5eaf457e30e27322e1a2716f356 Mon Sep 17 00:00:00 2001 From: tgravescs Date: Wed, 2 Oct 2013 12:54:17 -0500 Subject: [PATCH 33/34] Allow users to set the application name for Spark on Yarn --- docs/running-on-yarn.md | 1 + .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +- .../org/apache/spark/deploy/yarn/ClientArguments.scala | 6 ++++++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index c611db0af4..30128ec45d 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -50,6 +50,7 @@ The command to launch the YARN Client is as follows: --master-memory \ --worker-memory \ --worker-cores \ + --name \ --queue For example: 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 3362010106..076dd3c9b0 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 @@ -106,7 +106,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl logInfo("Setting up application submission context for ASM") val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) appContext.setApplicationId(appId) - appContext.setApplicationName("Spark") + appContext.setApplicationName(args.appName) return appContext } 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 cd651904d2..6b1ee94cb4 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 @@ -32,6 +32,7 @@ class ClientArguments(val args: Array[String]) { var numWorkers = 2 var amQueue = System.getProperty("QUEUE", "default") var amMemory: Int = 512 + var appName: String = "Spark" // TODO var inputFormatInfo: List[InputFormatInfo] = null @@ -78,6 +79,10 @@ class ClientArguments(val args: Array[String]) { amQueue = value args = tail + case ("--name") :: value :: tail => + appName = value + args = tail + case Nil => if (userJar == null || userClass == null) { printUsageAndExit(1) @@ -108,6 +113,7 @@ class ClientArguments(val args: Array[String]) { " --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\n" + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')" ) System.exit(exitCode) From c021b8c2028b9fd3851388e1d0c28729ca1eb2a5 Mon Sep 17 00:00:00 2001 From: tgravescs Date: Thu, 3 Oct 2013 08:07:19 -0500 Subject: [PATCH 34/34] Add default value to usage statement --- .../scala/org/apache/spark/deploy/yarn/ClientArguments.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 6b1ee94cb4..c56dbd99ba 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 @@ -113,7 +113,7 @@ class ClientArguments(val args: Array[String]) { " --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\n" + + " --name NAME The name of your application (Default: Spark)\n" + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')" ) System.exit(exitCode)