From 0dc1e2d60f89f07f54e0985d37cdcd32ad388f6a Mon Sep 17 00:00:00 2001 From: unknown Date: Mon, 22 Apr 2013 09:22:45 -0600 Subject: [PATCH 001/419] Examaple of cumulative counting using updateStateByKey --- ...kWordCumulativeCountUpdateStateByKey.scala | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala new file mode 100644 index 0000000000..db62246387 --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala @@ -0,0 +1,63 @@ +package spark.streaming.examples + +import spark.streaming._ +import spark.streaming.StreamingContext._ + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Usage: NetworkWordCumulativeCountUpdateStateByKey + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and describe the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run spark.streaming.examples.NetworkWordCumulativeCountUpdateStateByKey local[2] localhost 9999` + */ +object NetworkWordCumulativeCountUpdateStateByKey { + private def className[A](a: A)(implicit m: Manifest[A]) = m.toString + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: NetworkWordCountUpdateStateByKey \n" + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + + val updateFunc = (values: Seq[Int], state: Option[Int]) => { + val currentCount = values.foldLeft(0)(_ + _) + //println("currentCount: " + currentCount) + + val previousCount = state.getOrElse(0) + //println("previousCount: " + previousCount) + + val cumulative = Some(currentCount + previousCount) + //println("Cumulative: " + cumulative) + + cumulative + } + + // Create the context with a 10 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(10), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + ssc.checkpoint(".") + + // Create a NetworkInputDStream on target ip:port and count the + // words in input stream of \n delimited test (eg. generated by 'nc') + val lines = ssc.socketTextStream(args(1), args(2).toInt) + val words = lines.flatMap(_.split(" ")) + val wordDstream = words.map(x => (x, 1)) + + // Update the cumulative count using updateStateByKey + // This will give a Dstream made of state (which is the cumulative count of the words) + val stateDstream = wordDstream.updateStateByKey[Int](updateFunc) + + stateDstream.foreach(rdd => { + rdd.foreach(rddVal => { + println("Current Count: " + rddVal) + }) + }) + + ssc.start() + } +} From 1d54401d7e41095d8cbeeefd42c9d39ee500cd9f Mon Sep 17 00:00:00 2001 From: unknown Date: Tue, 30 Apr 2013 23:01:32 -0600 Subject: [PATCH 002/419] Modified as per TD's suggestions --- ...y.scala => StatefulNetworkWordCount.scala} | 23 +++++-------------- 1 file changed, 6 insertions(+), 17 deletions(-) rename examples/src/main/scala/spark/streaming/examples/{NetworkWordCumulativeCountUpdateStateByKey.scala => StatefulNetworkWordCount.scala} (71%) diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala similarity index 71% rename from examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala rename to examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index db62246387..b662cb1162 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -5,36 +5,31 @@ import spark.streaming.StreamingContext._ /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - * Usage: NetworkWordCumulativeCountUpdateStateByKey + * Usage: StatefulNetworkWordCount * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. * and describe the TCP server that Spark Streaming would connect to receive data. * * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run spark.streaming.examples.NetworkWordCumulativeCountUpdateStateByKey local[2] localhost 9999` + * `$ ./run spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ -object NetworkWordCumulativeCountUpdateStateByKey { +object StatefulNetworkWordCount { private def className[A](a: A)(implicit m: Manifest[A]) = m.toString def main(args: Array[String]) { if (args.length < 3) { - System.err.println("Usage: NetworkWordCountUpdateStateByKey \n" + + System.err.println("Usage: StatefulNetworkWordCount \n" + "In local mode, should be 'local[n]' with n > 1") System.exit(1) } val updateFunc = (values: Seq[Int], state: Option[Int]) => { val currentCount = values.foldLeft(0)(_ + _) - //println("currentCount: " + currentCount) val previousCount = state.getOrElse(0) - //println("previousCount: " + previousCount) - val cumulative = Some(currentCount + previousCount) - //println("Cumulative: " + cumulative) - - cumulative + Some(currentCount + previousCount) } // Create the context with a 10 second batch size @@ -51,13 +46,7 @@ object NetworkWordCumulativeCountUpdateStateByKey { // Update the cumulative count using updateStateByKey // This will give a Dstream made of state (which is the cumulative count of the words) val stateDstream = wordDstream.updateStateByKey[Int](updateFunc) - - stateDstream.foreach(rdd => { - rdd.foreach(rddVal => { - println("Current Count: " + rddVal) - }) - }) - + stateDstream.print() ssc.start() } } From cbf6a5ee1e7d290d04a0c5dac78d360266d415a4 Mon Sep 17 00:00:00 2001 From: unknown Date: Mon, 6 May 2013 08:05:45 -0600 Subject: [PATCH 003/419] Removed unused code, clarified intent of the program, batch size to 1 second --- .../streaming/examples/StatefulNetworkWordCount.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index b662cb1162..51c3c9f9b4 100644 --- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -4,7 +4,7 @@ import spark.streaming._ import spark.streaming.StreamingContext._ /** - * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every second. * Usage: StatefulNetworkWordCount * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. * and describe the TCP server that Spark Streaming would connect to receive data. @@ -15,8 +15,6 @@ import spark.streaming.StreamingContext._ * `$ ./run spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ object StatefulNetworkWordCount { - private def className[A](a: A)(implicit m: Manifest[A]) = m.toString - def main(args: Array[String]) { if (args.length < 3) { System.err.println("Usage: StatefulNetworkWordCount \n" + @@ -32,8 +30,8 @@ object StatefulNetworkWordCount { Some(currentCount + previousCount) } - // Create the context with a 10 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(10), + // Create the context with a 1 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(1), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) ssc.checkpoint(".") From 75d78c7ac90ce717bf1009ec4d335fb4a6cfde24 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 19 Jun 2013 11:18:42 -0500 Subject: [PATCH 004/419] Add support for Spark on Yarn on a secure Hadoop cluster --- .../scala/spark/deploy/SparkHadoopUtil.scala | 13 +---- .../spark/deploy/yarn/ApplicationMaster.scala | 17 ------- .../scala/spark/deploy/yarn/Client.scala | 50 ++++++++++++++++--- .../spark/deploy/yarn/WorkerRunnable.scala | 33 ++++++++++-- .../main/scala/spark/PairRDDFunctions.scala | 5 ++ core/src/main/scala/spark/SparkContext.scala | 9 ++++ .../spark/scheduler/InputFormatInfo.scala | 9 ++++ .../streaming/PairDStreamFunctions.scala | 7 ++- 8 files changed, 102 insertions(+), 41 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala index ab1ab9d8a7..b96c047e10 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala @@ -27,18 +27,7 @@ object SparkHadoopUtil { } def runAsUser(func: (Product) => Unit, args: Product, user: String) { - - // println("running as user " + jobUserName) - - UserGroupInformation.setConfiguration(yarnConf) - val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(user) - appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] { - def run: AnyRef = { - func(args) - // no return value ... - null - } - }) + func(args) } // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true. diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index aa72c1e5fe..f19648ec68 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -30,23 +30,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e def run() { - // Initialization - val jobUserName = Utils.getUserNameFromEnvironment() - logInfo("running as user " + jobUserName) - - // run as user ... - UserGroupInformation.setConfiguration(yarnConf) - val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(jobUserName) - appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] { - def run: AnyRef = { - runImpl() - return null - } - }) - } - - private def runImpl() { - appAttemptId = getApplicationAttemptId() resourceManager = registerWithResourceManager() val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala index 7a881e26df..f20cc31c7c 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -1,9 +1,13 @@ package spark.deploy.yarn import java.net.{InetSocketAddress, URI} +import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.mapred.Master import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ @@ -23,6 +27,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl var rpc: YarnRPC = YarnRPC.create(conf) val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + val credentials = UserGroupInformation.getCurrentUser().getCredentials(); def run() { init(yarnConf) @@ -41,7 +46,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl appContext.setQueue(args.amQueue) appContext.setAMContainerSpec(amContainer) appContext.setUser(args.amUser) - + submitApp(appContext) monitorApplication(appId) @@ -62,14 +67,21 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() - logInfo("Max mem capabililty of resources in this cluster " + maxMem) + logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) - // If the cluster does not have enough memory resources, exit. - val requestedMem = (args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + args.numWorkers * args.workerMemory - if (requestedMem > maxMem) { - logError("Cluster cannot satisfy memory resource request of " + requestedMem) + // if we have requested more then the clusters max for a single resource then exit. + if (args.workerMemory > maxMem) { + logError("the worker size is to large to run on this cluster " + args.workerMemory); System.exit(1) } + val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD + if (amMem > maxMem) { + logError("AM size is to large to run on this cluster " + amMem) + System.exit(1) + } + + // We could add checks to make sure the entire cluster has enough resources but that involves getting + // all the node reports and computing ourselves } def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { @@ -86,6 +98,15 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Upload Spark and the application JAR to the remote file system // Add them as local resources to the AM val fs = FileSystem.get(conf) + + val delegTokenRenewer = Master.getMasterPrincipal(conf); + if (UserGroupInformation.isSecurityEnabled()) { + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + logError("Can't get Master Kerberos principal for use as renewer") + System.exit(1) + } + } + Map("spark.jar" -> System.getenv("SPARK_JAR"), "app.jar" -> args.userJar, "log4j.properties" -> System.getenv("SPARK_LOG4J_CONF")) .foreach { case(destName, _localPath) => val localPath: String = if (_localPath != null) _localPath.trim() else "" @@ -97,6 +118,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl fs.copyFromLocalFile(false, true, src, dst) val destStatus = fs.getFileStatus(dst) + // get tokens for anything we upload to hdfs + if (UserGroupInformation.isSecurityEnabled()) { + fs.addDelegationTokens(delegTokenRenewer, credentials); + } + val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] amJarRsrc.setType(LocalResourceType.FILE) amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION) @@ -106,6 +132,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl locaResources(destName) = amJarRsrc } } + UserGroupInformation.getCurrentUser().addCredentials(credentials); return locaResources } @@ -115,6 +142,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val env = new HashMap[String, String]() Apps.addToEnvironment(env, Environment.USER.name, args.amUser) + // set this so that UGI set to correct user in unsecure mode + Apps.addToEnvironment(env, "HADOOP_USER_NAME", args.amUser) // If log4j present, ensure ours overrides all others if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./") @@ -195,7 +224,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } // Command for the ApplicationMaster - val commands = List[String]("java " + + val commands = List[String](Environment.JAVA_HOME.$() + "/bin/java " + " -server " + JAVA_OPTS + " spark.deploy.yarn.ApplicationMaster" + @@ -214,7 +243,12 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Memory for the ApplicationMaster capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) amContainer.setResource(capability) - + + // Setup security tokens + val dob = new DataOutputBuffer() + credentials.writeTokenStorageToStream(dob) + amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) + return amContainer } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala index a2bf0af762..e22d256a84 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -1,9 +1,12 @@ package spark.deploy.yarn import java.net.URI +import java.nio.ByteBuffer +import java.security.PrivilegedExceptionAction import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ @@ -11,7 +14,7 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import scala.collection.JavaConversions._ @@ -76,7 +79,13 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S */ ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) - val commands = List[String]("java " + + + val credentials = UserGroupInformation.getCurrentUser().getCredentials() + val dob = new DataOutputBuffer() + credentials.writeTokenStorageToStream(dob) + ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) + + val commands = List[String](Environment.JAVA_HOME.$() + "/bin/java " + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state. @@ -145,6 +154,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S val env = new HashMap[String, String]() // should we add this ? Apps.addToEnvironment(env, Environment.USER.name, Utils.getUserNameFromEnvironment()) + // set this so that UGI set to correct user in unsecure mode + Apps.addToEnvironment(env, "HADOOP_USER_NAME", Utils.getUserNameFromEnvironment()) // If log4j present, ensure ours overrides all others if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) { @@ -165,7 +176,23 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort() val cmAddress = NetUtils.createSocketAddr(cmHostPortStr) logInfo("Connecting to ContainerManager at " + cmHostPortStr) - return rpc.getProxy(classOf[ContainerManager], cmAddress, conf).asInstanceOf[ContainerManager] + + // use doAs and remoteUser here so we can add the container token and not + // pollute the current users credentials with all of the individual container tokens + val user = UserGroupInformation.createRemoteUser(container.getId().toString()); + val containerToken = container.getContainerToken(); + if (containerToken != null) { + user.addToken(ProtoUtils.convertFromProtoFormat(containerToken, cmAddress)) + } + + val proxy = user + .doAs(new PrivilegedExceptionAction[ContainerManager] { + def run: ContainerManager = { + return rpc.getProxy(classOf[ContainerManager], + cmAddress, conf).asInstanceOf[ContainerManager] + } + }); + return proxy; } } diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 2b0e697337..cc13f5d9c2 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -18,6 +18,7 @@ import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, HadoopMapReduceUtil, TaskAttemptID, TaskAttemptContext} +import org.apache.hadoop.security.UserGroupInformation import spark.partial.BoundedDouble import spark.partial.PartialResult @@ -584,6 +585,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf(self.context.hadoopConfiguration)) { + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) conf.setOutputKeyClass(keyClass) conf.setOutputValueClass(valueClass) // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index bc05d08fd6..c3a56938b5 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} +import org.apache.hadoop.security.UserGroupInformation import org.apache.mesos.MesosNativeLibrary @@ -294,6 +295,10 @@ class SparkContext( valueClass: Class[V], minSplits: Int = defaultMinSplits ): RDD[(K, V)] = { + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) } @@ -306,6 +311,10 @@ class SparkContext( minSplits: Int = defaultMinSplits ) : RDD[(K, V)] = { val conf = new JobConf(hadoopConfiguration) + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) FileInputFormat.setInputPaths(conf, path) new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) } diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala index 287f731787..30a56d7135 100644 --- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala @@ -3,6 +3,7 @@ package spark.scheduler import spark.Logging import scala.collection.immutable.Set import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.conf.Configuration @@ -70,6 +71,10 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // This method does not expect failures, since validate has already passed ... private def prefLocsFromMapreduceInputFormat(): Set[SplitInfo] = { val conf = new JobConf(configuration) + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) FileInputFormat.setInputPaths(conf, path) val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] = @@ -89,6 +94,10 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // This method does not expect failures, since validate has already passed ... private def prefLocsFromMapredInputFormat(): Set[SplitInfo] = { val jobConf = new JobConf(configuration) + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = jobConf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) FileInputFormat.setInputPaths(jobConf, path) val instance: org.apache.hadoop.mapred.InputFormat[_, _] = diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 3ec922957d..20ee1d3c5d 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -14,6 +14,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.mapred.OutputFormat +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.conf.Configuration class PairDStreamFunctions[K: ClassManifest, V: ClassManifest](self: DStream[(K,V)]) @@ -470,7 +471,11 @@ extends Serializable { valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf - ) { + ) { + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) val saveFunc = (rdd: RDD[(K, V)], time: Time) => { val file = rddToFileName(prefix, suffix, time) rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, conf) From 071ff7efa17a30db57a54f517a3cfecf3463db9f Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 20 Jun 2013 17:53:23 -0700 Subject: [PATCH 005/419] Enable building a fat jar for the Spark REPL --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0ea23b446f..e3892371e0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -197,7 +197,7 @@ object SparkBuild extends Build { def replSettings = sharedSettings ++ Seq( name := "spark-repl", libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - ) + ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", From 93a1643405d7c1a1fffe8210130341f34d64ea72 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Fri, 21 Jun 2013 14:21:52 +0100 Subject: [PATCH 006/419] Allow other twitter authorizations than username/password --- .../scala/spark/streaming/StreamingContext.scala | 15 ++++++++++++++- .../streaming/dstream/TwitterInputDStream.scala | 14 ++++++-------- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index b8b60aab43..f97e47ada0 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import twitter4j.Status +import twitter4j.auth.{Authorization, BasicAuthorization} /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -372,8 +373,20 @@ class StreamingContext private ( password: String, filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[Status] = twitterStream(new BasicAuthorization(username, password), filters, storageLevel) + + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J authentication + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + twitterAuth: Authorization, + filters: Seq[String] = Nil, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): DStream[Status] = { - val inputStream = new TwitterInputDStream(this, username, password, filters, storageLevel) + val inputStream = new TwitterInputDStream(this, twitterAuth, filters, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index c697498862..0b01091a52 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -6,6 +6,7 @@ import storage.StorageLevel import twitter4j._ import twitter4j.auth.BasicAuthorization +import twitter4j.auth.Authorization /* A stream of Twitter statuses, potentially filtered by one or more keywords. * @@ -16,21 +17,19 @@ import twitter4j.auth.BasicAuthorization private[streaming] class TwitterInputDStream( @transient ssc_ : StreamingContext, - username: String, - password: String, + twitterAuth: Authorization, filters: Seq[String], storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - + override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(username, password, filters, storageLevel) + new TwitterReceiver(twitterAuth, filters, storageLevel) } } private[streaming] class TwitterReceiver( - username: String, - password: String, + twitterAuth: Authorization, filters: Seq[String], storageLevel: StorageLevel ) extends NetworkReceiver[Status] { @@ -40,8 +39,7 @@ class TwitterReceiver( protected override def onStart() { blockGenerator.start() - twitterStream = new TwitterStreamFactory() - .getInstance(new BasicAuthorization(username, password)) + twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) twitterStream.addListener(new StatusListener { def onStatus(status: Status) = { blockGenerator += status From 60fbf7e461ff23978ba2b56cf494aad70ffc0bf5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 16 Jun 2013 20:08:25 -0700 Subject: [PATCH 007/419] Partially working checkpoint --- .../scala/spark/deploy/master/Master.scala | 2 + .../scala/spark/deploy/worker/Worker.scala | 2 + .../scala/spark/storage/BlockManagerUI.scala | 45 +++++++++++++++++-- .../src/main/scala/spark/util/AkkaUtils.scala | 36 +++++++-------- 4 files changed, 63 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3e965e82ac..edaacd87b8 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -57,6 +57,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def startWebUi() { val webUi = new MasterWebUI(context.system, self) + /* try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { @@ -64,6 +65,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logError("Failed to create web UI", e) System.exit(1) } + */ } override def receive = { diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 3dc2207170..f8fdab927e 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -89,6 +89,7 @@ private[spark] class Worker( def startWebUi() { val webUi = new WorkerWebUI(context.system, self, workDir) + /* try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { @@ -96,6 +97,7 @@ private[spark] class Worker( logError("Failed to create web UI", e) System.exit(1) } + */ } override def receive = { diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index e02281344a..f33855fad2 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -8,6 +8,12 @@ import cc.spray.Directives import spark.{Logging, SparkContext} import spark.util.AkkaUtils import spark.Utils +import spark.util.WebUI +import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler, ResourceHandler} +import org.eclipse.jetty.server.Handler +import xml.Elem +import xml.Node +import java.net.URLClassLoader /** @@ -32,7 +38,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, /** Start a HTTP server to run the Web interface */ def start() { try { - AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, handler, "BlockManagerHTTPServer") + AkkaUtils.startJettyServer("0.0.0.0", port, handlers) logInfo("Started BlockManager web UI at http://%s:%d".format(host, port)) } catch { case e: Exception => @@ -41,6 +47,40 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } } + val staticHandler = new ResourceHandler + println("RESOURCE: ") + val resource = getClass.getClassLoader.getResource(STATIC_RESOURCE_DIR) + staticHandler.setResourceBase(resource.toString) + + + val handlers = Array[(String, Handler)]( + ("/static", staticHandler), + ("*", WebUI.makeHandler(WebUI.makePage(indexContent, "Spark Storage"))) + ) + + def indexContent: Seq[Node] = { + val storageStatusList = sc.getExecutorStorageStatus + // Calculate macro-level statistics + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + +
+
+
    +
  • Memory: + {Utils.memoryBytesToString(maxMem - remainingMem)} Used + ({Utils.memoryBytesToString(remainingMem)} Available)
  • +
  • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
  • +
+
+
+
+ } + + /* val handler = { get { path("") { @@ -73,7 +113,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, getFromResourceDirectory(STATIC_RESOURCE_DIR) } } - } - + */ private[spark] def appUIAddress = "http://" + host + ":" + port } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index e93cc3b485..12581f535f 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -13,6 +13,9 @@ import cc.spray.io.pipelines.MessageHandlerDispatch.SingletonHandler import akka.dispatch.Await import spark.{Utils, SparkException} import java.util.concurrent.TimeoutException +import org.eclipse.jetty.server.Server +import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler} /** * Various utility classes for working with Akka. @@ -66,26 +69,21 @@ private[spark] object AkkaUtils { * handle requests. Returns the bound port or throws a SparkException on failure. * TODO: Not changing ip to host here - is it required ? */ - def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route, - name: String = "HttpServer"): ActorRef = { - val ioWorker = new IoWorker(actorSystem).start() - val httpService = actorSystem.actorOf(Props(new HttpService(route))) - val rootService = actorSystem.actorOf(Props(new SprayCanRootService(httpService))) - val server = actorSystem.actorOf( - Props(new HttpServer(ioWorker, SingletonHandler(rootService))), name = name) - actorSystem.registerOnTermination { ioWorker.stop() } - val timeout = 3.seconds - val future = server.ask(HttpServer.Bind(ip, port))(timeout) - try { - Await.result(future, timeout) match { - case bound: HttpServer.Bound => - return server - case other: Any => - throw new SparkException("Failed to bind web UI to port " + port + ": " + other) + def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]) = { + val handlersToRegister = handlers.map { case(path, handler) => + if (path == "*") { + handler + } else { + val contextHandler = new ContextHandler(path) + contextHandler.setHandler(handler) + contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] } - } catch { - case e: TimeoutException => - throw new SparkException("Failed to bind web UI to port " + port) } + + val handlerList = new HandlerList + handlerList.setHandlers(handlersToRegister) + val server = new Server(port) + server.setHandler(handlerList) + server.start() } } From e66f57019424bc5c7a1ca55cd335c34877b6d9cf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 13:53:13 -0700 Subject: [PATCH 008/419] Completely hacked version of block manager UI in jetty --- .../scala/spark/storage/BlockManagerUI.scala | 173 ++++++++++++++---- .../src/main/scala/spark/util/AkkaUtils.scala | 23 --- 2 files changed, 134 insertions(+), 62 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index f33855fad2..e577d1b597 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -11,9 +11,11 @@ import spark.Utils import spark.util.WebUI import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler, ResourceHandler} import org.eclipse.jetty.server.Handler +import javax.servlet.http.{HttpServletResponse, HttpServletRequest} import xml.Elem import xml.Node import java.net.URLClassLoader +import spark.util.WebUI._ /** @@ -38,7 +40,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, /** Start a HTTP server to run the Web interface */ def start() { try { - AkkaUtils.startJettyServer("0.0.0.0", port, handlers) + WebUI.startJettyServer("0.0.0.0", port, handlers) logInfo("Started BlockManager web UI at http://%s:%d".format(host, port)) } catch { case e: Exception => @@ -55,10 +57,109 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, val handlers = Array[(String, Handler)]( ("/static", staticHandler), - ("*", WebUI.makeHandler(WebUI.makePage(indexContent, "Spark Storage"))) + ("/rdd", (request: HttpServletRequest) => rddPage(request)), + ("*", (request: HttpServletRequest) => WebUI.makePage(indexPage, "Spark Storage")) ) - def indexContent: Seq[Node] = { + def rddPage(request: HttpServletRequest): Seq[Node] = { + val id = request.getParameter("id") + val prefix = "rdd_" + id.toString + val storageStatusList = sc.getExecutorStorageStatus + val filteredStorageStatusList = StorageUtils. + filterStorageStatusByPrefix(storageStatusList, prefix) + val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head + spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) + + val content = +
+
+
    +
  • + Storage Level: + {rddInfo.storageLevel.description} +
  • +
  • + Cached Partitions: + {rddInfo.numCachedPartitions} +
  • +
  • + Total Partitions: + {rddInfo.numPartitions} +
  • +
  • + Memory Size: + {Utils.memoryBytesToString(rddInfo.memSize)} +
  • +
  • + Disk Size: + {Utils.memoryBytesToString(rddInfo.diskSize)} +
  • +
+
+
+
+
+
+

RDD Summary

+
+ + + + + + + + + + + {storageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { case (k,v) => + + + + + + + } + } + +
Block NameStorage LevelSize in MemorySize on Disk
{k} + {v.storageLevel.description} + {Utils.memoryBytesToString(v.memSize)}{Utils.memoryBytesToString(v.diskSize)}
+
+
+
+
+
+

Worker Summary

+
+ + + + + + + + + + {for(status <- storageStatusList) { + + + + + + }} + +
HostMemory UsageDisk Usage
{status.blockManagerId.host + ":" + status.blockManagerId.port} + {Utils.memoryBytesToString(status.memUsed(prefix))} + ({Utils.memoryBytesToString(status.memRemaining)} Total Available) + {Utils.memoryBytesToString(status.diskUsed(prefix))}
+
+
; + + WebUI.makePage(content, "RDD Info: " + id) + } + + def indexPage: Seq[Node] = { val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) @@ -77,42 +178,36 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, -
- } - - /* - val handler = { - get { - path("") { - completeWith { - // Request the current storage status from the Master - val storageStatusList = sc.getExecutorStorageStatus - // Calculate macro-level statistics - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) - val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - spark.storage.html.index. - render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList) - } - } ~ - path("rdd") { - parameter("id") { id => - completeWith { - val prefix = "rdd_" + id.toString - val storageStatusList = sc.getExecutorStorageStatus - val filteredStorageStatusList = StorageUtils. - filterStorageStatusByPrefix(storageStatusList, prefix) - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) +
+ + + + + + + + + + + + + {for (rdd <- rdds) yield + + + + + + + + } - } - } ~ - pathPrefix("static") { - getFromResourceDirectory(STATIC_RESOURCE_DIR) - } - } - */ + +
RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
+ + {rdd.name} + + {rdd.storageLevel.description} + {rdd.numCachedPartitions}{rdd.numCachedPartitions / rdd.numPartitions.toDouble}{Utils.memoryBytesToString(rdd.memSize)}{Utils.memoryBytesToString(rdd.diskSize)}
+ } private[spark] def appUIAddress = "http://" + host + ":" + port } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 12581f535f..bd2d637ae7 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -63,27 +63,4 @@ private[spark] object AkkaUtils { val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get return (actorSystem, boundPort) } - - /** - * Creates a Spray HTTP server bound to a given IP and port with a given Spray Route object to - * handle requests. Returns the bound port or throws a SparkException on failure. - * TODO: Not changing ip to host here - is it required ? - */ - def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]) = { - val handlersToRegister = handlers.map { case(path, handler) => - if (path == "*") { - handler - } else { - val contextHandler = new ContextHandler(path) - contextHandler.setHandler(handler) - contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] - } - } - - val handlerList = new HandlerList - handlerList.setHandlers(handlersToRegister) - val server = new Server(port) - server.setHandler(handlerList) - server.start() - } } From 7cd70dc2c19a5c95a214786f9b0712a42de6cba5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 13:58:35 -0700 Subject: [PATCH 009/419] Minor cleanup --- .../scala/spark/storage/BlockManagerUI.scala | 112 +++++++++--------- 1 file changed, 55 insertions(+), 57 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index e577d1b597..9e1220c92a 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -49,16 +49,10 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } } - val staticHandler = new ResourceHandler - println("RESOURCE: ") - val resource = getClass.getClassLoader.getResource(STATIC_RESOURCE_DIR) - staticHandler.setResourceBase(resource.toString) - - val handlers = Array[(String, Handler)]( - ("/static", staticHandler), + ("/static", createStaticHandler(STATIC_RESOURCE_DIR)), ("/rdd", (request: HttpServletRequest) => rddPage(request)), - ("*", (request: HttpServletRequest) => WebUI.makePage(indexPage, "Spark Storage")) + ("*", (request: HttpServletRequest) => indexPage) ) def rddPage(request: HttpServletRequest): Seq[Node] = { @@ -112,16 +106,17 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, - {storageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { case (k,v) => - - {k} - - {v.storageLevel.description} - - {Utils.memoryBytesToString(v.memSize)} - {Utils.memoryBytesToString(v.diskSize)} - - } + {filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { + case (k,v) => + + {k} + + {v.storageLevel.description} + + {Utils.memoryBytesToString(v.memSize)} + {Utils.memoryBytesToString(v.diskSize)} + + } } @@ -168,46 +163,49 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, .reduceOption(_+_).getOrElse(0L) val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) -
-
-
    -
  • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
  • -
  • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
  • -
+ val content = +
+
+
    +
  • Memory: + {Utils.memoryBytesToString(maxMem - remainingMem)} Used + ({Utils.memoryBytesToString(remainingMem)} Available)
  • +
  • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
  • +
+
-
-
- - - - - - - - - - - - - {for (rdd <- rdds) yield - - - - - - - - - } - -
RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
- - {rdd.name} - - {rdd.storageLevel.description} - {rdd.numCachedPartitions}{rdd.numCachedPartitions / rdd.numPartitions.toDouble}{Utils.memoryBytesToString(rdd.memSize)}{Utils.memoryBytesToString(rdd.diskSize)}
+
+ + + + + + + + + + + + + {for (rdd <- rdds) yield + + + + + + + + + } + +
RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
+ + {rdd.name} + + {rdd.storageLevel.description} + {rdd.numCachedPartitions}{rdd.numCachedPartitions / rdd.numPartitions.toDouble}{Utils.memoryBytesToString(rdd.memSize)}{Utils.memoryBytesToString(rdd.diskSize)}
; + + WebUI.makePage(content, "Spark Storage") } private[spark] def appUIAddress = "http://" + host + ":" + port } From 950f83535a8393a131f97a1a64454a446c5ea31c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 14:27:11 -0700 Subject: [PATCH 010/419] Adding deterministic port --- .../scala/spark/storage/BlockManagerUI.scala | 22 +++++++++---------- 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 9e1220c92a..cea5c0907e 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -25,23 +25,16 @@ private[spark] class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, sc: SparkContext) extends Directives with Logging { - val STATIC_RESOURCE_DIR = "spark/deploy/static" - implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() - val port = if (System.getProperty("spark.ui.port") != null) { - System.getProperty("spark.ui.port").toInt - } else { - // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which - // random port it bound to, so we have to try to find a local one by creating a socket. - Utils.findFreePort() - } + val port = Option(System.getProperty("spark.ui.port")) + .getOrElse(BlockManagerUI.DEFAULT_PORT).toInt /** Start a HTTP server to run the Web interface */ def start() { try { - WebUI.startJettyServer("0.0.0.0", port, handlers) - logInfo("Started BlockManager web UI at http://%s:%d".format(host, port)) + val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started BlockManager web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create BlockManager WebUI", e) @@ -50,7 +43,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } val handlers = Array[(String, Handler)]( - ("/static", createStaticHandler(STATIC_RESOURCE_DIR)), + ("/static", createStaticHandler(BlockManagerUI.STATIC_RESOURCE_DIR)), ("/rdd", (request: HttpServletRequest) => rddPage(request)), ("*", (request: HttpServletRequest) => indexPage) ) @@ -209,3 +202,8 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } private[spark] def appUIAddress = "http://" + host + ":" + port } + +object BlockManagerUI { + val STATIC_RESOURCE_DIR = "spark/deploy/static" + val DEFAULT_PORT = "33000" +} From 7e6977b6c5f7c889b189285a7fa765db5ff2a285 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 14:59:57 -0700 Subject: [PATCH 011/419] Fix in storage status page --- .../scala/spark/storage/BlockManagerUI.scala | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index cea5c0907e..6ac4398de7 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -129,16 +129,18 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, - {for(status <- storageStatusList) { - - {status.blockManagerId.host + ":" + status.blockManagerId.port} - - {Utils.memoryBytesToString(status.memUsed(prefix))} - ({Utils.memoryBytesToString(status.memRemaining)} Total Available) - - {Utils.memoryBytesToString(status.diskUsed(prefix))} - - }} + {filteredStorageStatusList.map { + status => + + {status.blockManagerId.host + ":" + status.blockManagerId.port} + + {Utils.memoryBytesToString(status.memUsed(prefix))} + ({Utils.memoryBytesToString(status.memRemaining)} Total Available) + + {Utils.memoryBytesToString(status.diskUsed(prefix))} + + } + }
From 2c36a514aac2c15822c67f536c9f50450c8d5761 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 21:44:26 -0700 Subject: [PATCH 012/419] Spray refactoring for master web UI --- .../scala/spark/deploy/master/Master.scala | 12 +- .../spark/deploy/master/MasterWebUI.scala | 205 +++++++++++++++++- 2 files changed, 201 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index edaacd87b8..6a7bbdfcbf 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -56,16 +56,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } def startWebUi() { - val webUi = new MasterWebUI(context.system, self) - /* - try { - AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) - } catch { - case e: Exception => - logError("Failed to create web UI", e) - System.exit(1) - } - */ + val webUi = new MasterWebUI(self) + webUi.start() } override def receive = { diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index a4e21c8130..6393f2244b 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -10,20 +10,207 @@ import cc.spray.directives._ import cc.spray.typeconversion.TwirlSupport._ import cc.spray.http.MediaTypes import cc.spray.typeconversion.SprayJsonSupport._ +import scala.xml.Node +import spark.{Logging, Utils} +import spark.util.{WebUI => UtilsWebUI} import spark.deploy._ import spark.deploy.JsonProtocol._ +import concurrent.Future +import org.eclipse.jetty.server.Handler +import spark.util.WebUI._ +import spark.deploy.MasterState +import javax.servlet.http.HttpServletRequest +import java.util.concurrent.TimeUnit /** * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { - val RESOURCE_DIR = "spark/deploy/master/webui" - val STATIC_RESOURCE_DIR = "spark/deploy/static" - - implicit val timeout = Timeout(Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) - +class MasterWebUI(master: ActorRef) extends Logging { + + implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val host = Utils.localHostName() + val port = Option(System.getProperty("spark.ui.port")) + .getOrElse(MasterWebUI.DEFAULT_PORT).toInt + + def start() { + try { + val (server, boundPort) = UtilsWebUI.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Master WebUI", e) + System.exit(1) + } + } + + val handlers = Array[(String, Handler)]( + ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), + ("*", (request: HttpServletRequest) => index) + ) + + def appDetail(request: HttpServletRequest): Seq[Node] = { + val appId = request.getParameter("appId") + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + val app = state.activeApps.find(_.id == appId).getOrElse({ + state.completedApps.find(_.id == appId).getOrElse(null) + }) + val content = +
+
+
    +
  • ID: app.id
  • +
  • Description: app.desc.name
  • +
  • User: app.desc.user
  • +
  • Cores: + {if(app.desc.maxCores == Integer.MAX_VALUE) { + "Unlimited %s granted".format(app.coresGranted) + } else { + "%s (%s granted, %s left)".format( + app.desc.maxCores, + app.coresGranted, + app.coresLeft) } + } + } +
  • +
  • Memory per Slave: @app.desc.memoryPerSlave
  • +
  • Submit Date: @app.submitDate
  • +
  • State: @app.state
  • +
  • Application Detail UI
  • +
+
+
+ +
+ + +
+
+

Executor Summary

+
+ @executors_table(app.executors.values.toList) +
+
+ + } + + def index: Seq[Node] = { + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + + val content = +
+
+
    +
  • URL:{state.uri}
  • +
  • Workers:{state.workers.size}
  • +
  • Cores: {state.workers.map(_.cores).sum}Total, + {state.workers.map(_.coresUsed).sum} Used
  • +
  • Memory: {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, + {Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
  • +
  • Applications: {state.activeApps.size} Running, {state.completedApps.size} Completed
  • +
+
+
+ +
+
+

Workers

+
+ {workerTable(state.workers.sortBy(_.id))} +
+
+ +
+ +
+
+

Running Applications

+
+ {appTable(state.activeApps.sortBy(_.startTime).reverse)} +
+
+ +
+ +
+
+

Completed Applications

+
+ {appTable(state.completedApps.sortBy(_.endTime).reverse)} +
+
; + UtilsWebUI.makePage(content, "Spark Master: " + state.uri) + } + + def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { + + + + + + + + + + + + {workers.map{ worker => + + + + + + + + } + } + +
IDAddressStateCoresMemory
+ {worker.id} + {worker.host}:{worker.port}{worker.state}{worker.cores} ({worker.coresUsed} Used){Utils.memoryMegabytesToString(worker.memory)} + ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used)
+ } + + def appTable(apps: Seq[spark.deploy.master.ApplicationInfo]) = { + + + + + + + + + + + + + + + {apps.map{app => + + + + + + + + + + + } + } + +
IDDescriptionCoresMemory per NodeSubmit TimeUserStateDuration
+ {app.id} + {app.desc.name} + {app.coresGranted} + {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)}{WebUI.formatDate(app.submitDate)}{app.desc.user}{app.state.toString}{WebUI.formatDuration(app.duration)}
+ } + + /* val handler = { get { (path("") & parameters('format ?)) { @@ -71,4 +258,10 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct getFromResourceDirectory(RESOURCE_DIR) } } + */ } + +object MasterWebUI { + val STATIC_RESOURCE_DIR = "spark/deploy/static" + val DEFAULT_PORT = "34000" +} \ No newline at end of file From f85fd7a793e6b4c023075f962ffcb0893d622fc3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 21:58:17 -0700 Subject: [PATCH 013/419] Commenting unfinished part --- core/src/main/scala/spark/deploy/master/MasterWebUI.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 6393f2244b..c76b5b1e0e 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -78,7 +78,7 @@ class MasterWebUI(master: ActorRef) extends Logging {
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • State: @app.state
  • -
  • Application Detail UI
  • +
  • Application Detail UI
  • @@ -90,7 +90,7 @@ class MasterWebUI(master: ActorRef) extends Logging {

    Executor Summary


    - @executors_table(app.executors.values.toList) + @executors_table(app.executors.values.toList)
    From e55cf0245fa7d352f948312bb81cf2c09fe63d6f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 21:59:25 -0700 Subject: [PATCH 014/419] Adding WebUI file --- core/src/main/scala/spark/util/WebUI.scala | 104 +++++++++++++++++++++ 1 file changed, 104 insertions(+) create mode 100644 core/src/main/scala/spark/util/WebUI.scala diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala new file mode 100644 index 0000000000..eaacd95691 --- /dev/null +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -0,0 +1,104 @@ +package spark.util + +import xml.Elem +import xml.Node +import util.parsing.json.{JSONFormat, JSONObject} +import org.eclipse.jetty.server.{Server, Request, Handler} +import javax.servlet.http.{HttpServletResponse, HttpServletRequest} +import org.eclipse.jetty.util.component.LifeCycle.Listener +import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} +import util.Try +import util.Success +import util.Failure +import spark.Logging +import annotation.tailrec + +object WebUI extends Logging { + type Responder[T] = HttpServletRequest => T + + implicit def jsonResponderToHandler(responder: Responder[JSONObject]): Handler = { + createHandler(responder, "text/json") + } + + implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = { + createHandler(responder, "text/html") + } + + def createHandler[T <% AnyRef](responder: Responder[T], contentType: String): Handler = { + new AbstractHandler { + def handle(target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse) { + response.setContentType("%s;charset=utf-8".format(contentType)) + response.setStatus(HttpServletResponse.SC_OK) + baseRequest.setHandled(true) + response.getWriter().println( + responder(request).toString + ) + } + } + } + + def createStaticHandler(resourceBase: String): ResourceHandler = { + val staticHandler = new ResourceHandler + val resource = getClass.getClassLoader.getResource(resourceBase) + staticHandler.setResourceBase(resource.toString) + staticHandler + } + + def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]): (Server, Int) = { + val handlersToRegister = handlers.map { case(path, handler) => + if (path == "*") { + handler + } else { + val contextHandler = new ContextHandler(path) + contextHandler.setHandler(handler) + contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] + } + } + + val handlerList = new HandlerList + handlerList.setHandlers(handlersToRegister) + + @tailrec + def connect(currentPort: Int): (Server, Int) = { + val server = new Server(port) + server.setHandler(handlerList) + Try { server.start() } match { + case s: Success[_] => (server, currentPort) + case f: Failure[_] => + logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) + connect((currentPort + 1) % 65536) + } + } + + connect(port) + } + + def makePage(content: => Seq[Node], title: String): Seq[Node] = { + + + + + + + {title} + + +
    +
    +
    + +

    + {title} +

    +
    +
    + {content} +
    + + + } +} From 9161db5478c3e8c1c124070bacdf571b744cee22 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 09:50:24 -0700 Subject: [PATCH 015/419] Cleaning up master web UI --- .../spark/deploy/master/MasterWebUI.scala | 203 ++++++++---------- 1 file changed, 94 insertions(+), 109 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index c76b5b1e0e..0b7d163980 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -1,27 +1,19 @@ package spark.deploy.master -import akka.actor.{ActorRef, ActorSystem} +import akka.actor.{ActorRef} import akka.dispatch.Await import akka.pattern.ask -import akka.util.{Duration, Timeout} +import akka.util.{Duration} import akka.util.duration._ -import cc.spray.Directives -import cc.spray.directives._ -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.http.MediaTypes -import cc.spray.typeconversion.SprayJsonSupport._ import scala.xml.Node import spark.{Logging, Utils} import spark.util.{WebUI => UtilsWebUI} import spark.deploy._ -import spark.deploy.JsonProtocol._ -import concurrent.Future import org.eclipse.jetty.server.Handler import spark.util.WebUI._ import spark.deploy.MasterState import javax.servlet.http.HttpServletRequest -import java.util.concurrent.TimeUnit /** * Web UI server for the standalone master. @@ -47,9 +39,11 @@ class MasterWebUI(master: ActorRef) extends Logging { val handlers = Array[(String, Handler)]( ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), + ("/app", (request: HttpServletRequest) => appDetail(request)), ("*", (request: HttpServletRequest) => index) ) + /** Executor details for a particular application */ def appDetail(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] @@ -65,37 +59,73 @@ class MasterWebUI(master: ActorRef) extends Logging {
  • Description: app.desc.name
  • User: app.desc.user
  • Cores: - {if(app.desc.maxCores == Integer.MAX_VALUE) { - "Unlimited %s granted".format(app.coresGranted) - } else { - "%s (%s granted, %s left)".format( - app.desc.maxCores, - app.coresGranted, - app.coresLeft) } - } - } -
  • -
  • Memory per Slave: @app.desc.memoryPerSlave
  • -
  • Submit Date: @app.submitDate
  • -
  • State: @app.state
  • -
  • Application Detail UI
  • - - - + { + if (app.desc.maxCores == Integer.MAX_VALUE) { + "Unlimited %s granted".format(app.coresGranted) + } else { + "%s (%s granted, %s left)".format( + app.desc.maxCores, app.coresGranted, app.coresLeft) + } + } + +
  • Memory per Slave: {app.desc.memoryPerSlave}
  • +
  • Submit Date: {app.submitDate}
  • +
  • State: {app.state}
  • +
  • Application Detail UI
  • + + +
    - +

    Executor Summary


    - @executors_table(app.executors.values.toList) + {executorsTable(app.executors.values.toList)}
    - + UtilsWebUI.makePage(content, "Application Info: " + app.desc.name) } + def executorsTable(executors: Seq[ExecutorInfo]): Seq[Node] = { + + + + + + + + + + + + + {executors.map(executorRow)} + +
    ExecutorIDWorkerCoresMemoryStateLogs
    + } + + def executorRow(executor: ExecutorInfo): Seq[Node] = { + + {executor.id} + + {executor.worker.id} + + {executor.cores} + {executor.memory} + {executor.state} + + stdout + stdout + + + } + + /** Index view listing applications and executors */ def index: Seq[Node] = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] val state = Await.result(stateFuture, 3 seconds) @@ -108,9 +138,12 @@ class MasterWebUI(master: ActorRef) extends Logging {
  • Workers:{state.workers.size}
  • Cores: {state.workers.map(_.cores).sum}Total, {state.workers.map(_.coresUsed).sum} Used
  • -
  • Memory: {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, +
  • Memory: + {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, {Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
  • -
  • Applications: {state.activeApps.size} Running, {state.completedApps.size} Completed
  • +
  • Applications: + {state.activeApps.size} Running, + {state.completedApps.size} Completed
  • @@ -157,17 +190,18 @@ class MasterWebUI(master: ActorRef) extends Logging { - {workers.map{ worker => - - - {worker.id} - - {worker.host}:{worker.port} - {worker.state} - {worker.cores} ({worker.coresUsed} Used) - {Utils.memoryMegabytesToString(worker.memory)} - ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) - + { + workers.map{ worker => + + + {worker.id} + + {worker.host}:{worker.port} + {worker.state} + {worker.cores} ({worker.coresUsed} Used) + {Utils.memoryMegabytesToString(worker.memory)} + ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) + } } @@ -189,76 +223,27 @@ class MasterWebUI(master: ActorRef) extends Logging { - {apps.map{app => - - - {app.id} - - {app.desc.name} - - {app.coresGranted} - - {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} - {WebUI.formatDate(app.submitDate)} - {app.desc.user} - {app.state.toString} - {WebUI.formatDuration(app.duration)} - + { + apps.map{ app => + + + {app.id} + + {app.desc.name} + + {app.coresGranted} + + {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} + {WebUI.formatDate(app.submitDate)} + {app.desc.user} + {app.state.toString} + {WebUI.formatDuration(app.duration)} + + } } - } } - - /* - val handler = { - get { - (path("") & parameters('format ?)) { - case Some(js) if js.equalsIgnoreCase("json") => - val future = master ? RequestMasterState - respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(future.mapTo[MasterState]) - } - case _ => - completeWith { - val future = master ? RequestMasterState - future.map { - masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState]) - } - } - } ~ - path("app") { - parameters("appId", 'format ?) { - case (appId, Some(js)) if (js.equalsIgnoreCase("json")) => - val future = master ? RequestMasterState - val appInfo = for (masterState <- future.mapTo[MasterState]) yield { - masterState.activeApps.find(_.id == appId).getOrElse({ - masterState.completedApps.find(_.id == appId).getOrElse(null) - }) - } - respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(appInfo.mapTo[ApplicationInfo]) - } - case (appId, _) => - completeWith { - val future = master ? RequestMasterState - future.map { state => - val masterState = state.asInstanceOf[MasterState] - val app = masterState.activeApps.find(_.id == appId).getOrElse({ - masterState.completedApps.find(_.id == appId).getOrElse(null) - }) - spark.deploy.master.html.app_details.render(app) - } - } - } - } ~ - pathPrefix("static") { - getFromResourceDirectory(STATIC_RESOURCE_DIR) - } ~ - getFromResourceDirectory(RESOURCE_DIR) - } - } - */ } object MasterWebUI { From ee73c09ac965076c9239587721ffa76741b99483 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 10:10:48 -0700 Subject: [PATCH 016/419] Some comments --- core/src/main/scala/spark/deploy/master/MasterWebUI.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 0b7d163980..34f50fd5e4 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -78,14 +78,13 @@ class MasterWebUI(master: ActorRef) extends Logging {
    - -
    +

    Executor Summary


    {executorsTable(app.executors.values.toList)}
    -
    +
    ; UtilsWebUI.makePage(content, "Application Info: " + app.desc.name) } From fc94576ece99f2b224a951b32f0f6360701b7cd3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 13:11:57 -0700 Subject: [PATCH 017/419] Adding worker version of UI --- .../spark/deploy/master/MasterWebUI.scala | 16 +-- .../scala/spark/deploy/worker/Worker.scala | 12 +- .../spark/deploy/worker/WorkerWebUI.scala | 135 +++++++++++++++++- core/src/main/scala/spark/util/WebUI.scala | 31 +++- 4 files changed, 162 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 34f50fd5e4..a2e9dfd762 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -20,10 +20,10 @@ import javax.servlet.http.HttpServletRequest */ private[spark] class MasterWebUI(master: ActorRef) extends Logging { - - implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + implicit val timeout = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() - val port = Option(System.getProperty("spark.ui.port")) + val port = Option(System.getProperty("master.ui.port")) .getOrElse(MasterWebUI.DEFAULT_PORT).toInt def start() { @@ -82,13 +82,13 @@ class MasterWebUI(master: ActorRef) extends Logging {

    Executor Summary


    - {executorsTable(app.executors.values.toList)} + {executorTable(app.executors.values.toList)}
    ; UtilsWebUI.makePage(content, "Application Info: " + app.desc.name) } - def executorsTable(executors: Seq[ExecutorInfo]): Seq[Node] = { + def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -119,7 +119,7 @@ class MasterWebUI(master: ActorRef) extends Logging { stdoutstdout + .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stderr } @@ -135,7 +135,7 @@ class MasterWebUI(master: ActorRef) extends Logging {
    • URL:{state.uri}
    • Workers:{state.workers.size}
    • -
    • Cores: {state.workers.map(_.cores).sum}Total, +
    • Cores: {state.workers.map(_.cores).sum} Total, {state.workers.map(_.coresUsed).sum} Used
    • Memory: {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, @@ -247,5 +247,5 @@ class MasterWebUI(master: ActorRef) extends Logging { object MasterWebUI { val STATIC_RESOURCE_DIR = "spark/deploy/static" - val DEFAULT_PORT = "34000" + val DEFAULT_PORT = "8080" } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index f8fdab927e..3878fe3f7b 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -88,16 +88,8 @@ private[spark] class Worker( } def startWebUi() { - val webUi = new WorkerWebUI(context.system, self, workDir) - /* - try { - AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) - } catch { - case e: Exception => - logError("Failed to create web UI", e) - System.exit(1) - } - */ + val webUi = new WorkerWebUI(self, workDir) + webUi.start() } override def receive = { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 3235c50d1b..b8b4b89738 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -8,22 +8,137 @@ import akka.util.duration._ import cc.spray.Directives import cc.spray.typeconversion.TwirlSupport._ import cc.spray.http.MediaTypes -import cc.spray.typeconversion.SprayJsonSupport._ import spark.deploy.{WorkerState, RequestWorkerState} import spark.deploy.JsonProtocol._ import java.io.File +import spark.util.{WebUI => UtilsWebUI} +import spark.{Utils, Logging} +import org.eclipse.jetty.server.Handler +import spark.util.WebUI._ +import spark.deploy.WorkerState +import scala.io.Source +import javax.servlet.http.HttpServletRequest +import xml.Node /** * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef, workDir: File) extends Directives { - val RESOURCE_DIR = "spark/deploy/worker/webui" - val STATIC_RESOURCE_DIR = "spark/deploy/static" - - implicit val timeout = Timeout(Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) - +class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { + implicit val timeout = Timeout( + Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) + val host = Utils.localHostName() + val port = Option(System.getProperty("wroker.ui.port")) + .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt + + val handlers = Array[(String, Handler)]( + ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), + ("/log", (request: HttpServletRequest) => log(request)), + ("*", (request: HttpServletRequest) => index) + ) + + def start() { + try { + val (server, boundPort) = UtilsWebUI.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Worker WebUI", e) + System.exit(1) + } + } + + def index(): Seq[Node] = { + val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val workerState = Await.result(stateFuture, 3 seconds) + val content = +
      +
      +
        +
      • ID: {workerState.workerId}
      • +
      • + Master URL: {workerState.masterUrl} +
      • +
      • Cores: {workerState.cores} ({workerState.coresUsed} Used)
      • +
      • Memory: {Utils.memoryMegabytesToString(workerState.memory)} + ({Utils.memoryMegabytesToString(workerState.memoryUsed)} Used)
      • +
      +

      Back to Master

      +
      +
      +
      + +
      +
      +

      Running Executors {workerState.executors.size}

      +
      + {executorTable(workerState.executors)} +
      +
      +
      + +
      +
      +

      Finished Executors

      +
      + {executorTable(workerState.finishedExecutors)} +
      +
      ; + + UtilsWebUI.makePage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + } + + def executorTable(executors: Seq[ExecutorRunner]): Seq[Node] = { +
    + + + + + + + + + + + {executors.map(executorRow)} + +
    ExecutorIDCoresMemoryJob DetailsLogs
    + } + + def executorRow(executor: ExecutorRunner): Seq[Node] = { + + {executor.execId} + {executor.cores} + {Utils.memoryMegabytesToString(executor.memory)} + +
      +
    • ID: {executor.appId}
    • +
    • Name: {executor.appDesc.name}
    • +
    • User: {executor.appDesc.user}
    • +
    + + + stdout + stderr + + + } + + def log(request: HttpServletRequest): String = { + val appId = request.getParameter("appId") + val executorId = request.getParameter("executorId") + val logType = request.getParameter("logType") + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) + val source = Source.fromFile(path) + val lines = source.mkString + source.close() + lines + } + + /* val handler = { get { (path("") & parameters('format ?)) { @@ -54,4 +169,10 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef, workDir: File) getFromResourceDirectory(RESOURCE_DIR) } } + */ +} + +object WorkerWebUI { + val STATIC_RESOURCE_DIR = "spark/deploy/static" + val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index eaacd95691..34b776f1d8 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -16,13 +16,14 @@ import annotation.tailrec object WebUI extends Logging { type Responder[T] = HttpServletRequest => T - implicit def jsonResponderToHandler(responder: Responder[JSONObject]): Handler = { + implicit def jsonResponderToHandler(responder: Responder[JSONObject]): Handler = createHandler(responder, "text/json") - } - implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = { + implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = createHandler(responder, "text/html") - } + + implicit def textResponderToHandler(responder: Responder[String]): Handler = + createHandler(responder, "text/plain") def createHandler[T <% AnyRef](responder: Responder[T], contentType: String): Handler = { new AbstractHandler { @@ -40,13 +41,29 @@ object WebUI extends Logging { } } + /** Create and return a staticHandler if resourceBase can be located */ def createStaticHandler(resourceBase: String): ResourceHandler = { val staticHandler = new ResourceHandler - val resource = getClass.getClassLoader.getResource(resourceBase) - staticHandler.setResourceBase(resource.toString) - staticHandler + Option(getClass.getClassLoader.getResource(resourceBase)) match { + case Some(res) => + staticHandler.setResourceBase (res.toString) + staticHandler + } } + /* + /** Create and return a staticHandler if resourceBase can be located */ + def createStaticHandler(resourceBase: String): Option[ResourceHandler] = { + val staticHandler = new ResourceHandler + Option(getClass.getClassLoader.getResource(resourceBase)) match { + case Some(res) => + staticHandler.setResourceBase (res.toString) + Some(staticHandler) + case None => None + } + } + */ + def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]): (Server, Int) = { val handlersToRegister = handlers.map { case(path, handler) => if (path == "*") { From 91ec5a1a04339983d57a72d8df8f1d769d8d855a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 15:13:12 -0700 Subject: [PATCH 018/419] Changing JSON protocol and removing spray code --- .../scala/spark/deploy/JsonProtocol.scala | 116 ++++++++---------- .../spark/deploy/master/MasterWebUI.scala | 6 +- .../spark/deploy/worker/WorkerWebUI.scala | 6 +- .../scala/spark/storage/BlockManagerUI.scala | 11 +- .../src/main/scala/spark/util/AkkaUtils.scala | 15 +-- core/src/main/scala/spark/util/WebUI.scala | 12 +- project/SparkBuild.scala | 7 +- project/plugins.sbt | 2 - 8 files changed, 67 insertions(+), 108 deletions(-) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index ea832101d2..b4365d31e9 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -1,79 +1,65 @@ package spark.deploy import master.{ApplicationInfo, WorkerInfo} +import net.liftweb.json.JsonDSL._ import worker.ExecutorRunner -import cc.spray.json._ -/** - * spray-json helper class containing implicit conversion to json for marshalling responses - */ -private[spark] object JsonProtocol extends DefaultJsonProtocol { - implicit object WorkerInfoJsonFormat extends RootJsonWriter[WorkerInfo] { - def write(obj: WorkerInfo) = JsObject( - "id" -> JsString(obj.id), - "host" -> JsString(obj.host), - "port" -> JsNumber(obj.port), - "webuiaddress" -> JsString(obj.webUiAddress), - "cores" -> JsNumber(obj.cores), - "coresused" -> JsNumber(obj.coresUsed), - "memory" -> JsNumber(obj.memory), - "memoryused" -> JsNumber(obj.memoryUsed) - ) +object JsonProtocol { + def writeWorkerInfo(obj: WorkerInfo) = { + ("id" -> obj.id) ~ + ("host" -> obj.host) ~ + ("port" -> obj.port) ~ + ("webuiaddress" -> obj.webUiAddress) ~ + ("cores" -> obj.cores) ~ + ("coresused" -> obj.coresUsed) ~ + ("memory" -> obj.memory) ~ + ("memoryused" -> obj.memoryUsed) + } + + def writeApplicationInfo(obj: ApplicationInfo) = { + ("starttime" -> obj.startTime) ~ + ("id" -> obj.id) ~ + ("name" -> obj.desc.name) ~ + ("cores" -> obj.desc.maxCores) ~ + ("user" -> obj.desc.user) ~ + ("memoryperslave" -> obj.desc.memoryPerSlave) ~ + ("submitdate" -> obj.submitDate.toString) } - implicit object AppInfoJsonFormat extends RootJsonWriter[ApplicationInfo] { - def write(obj: ApplicationInfo) = JsObject( - "starttime" -> JsNumber(obj.startTime), - "id" -> JsString(obj.id), - "name" -> JsString(obj.desc.name), - "cores" -> JsNumber(obj.desc.maxCores), - "user" -> JsString(obj.desc.user), - "memoryperslave" -> JsNumber(obj.desc.memoryPerSlave), - "submitdate" -> JsString(obj.submitDate.toString)) + def writeApplicationDescription(obj: ApplicationDescription) = { + ("name" -> obj.name) ~ + ("cores" -> obj.maxCores) ~ + ("memoryperslave" -> obj.memoryPerSlave) ~ + ("user" -> obj.user) } - implicit object AppDescriptionJsonFormat extends RootJsonWriter[ApplicationDescription] { - def write(obj: ApplicationDescription) = JsObject( - "name" -> JsString(obj.name), - "cores" -> JsNumber(obj.maxCores), - "memoryperslave" -> JsNumber(obj.memoryPerSlave), - "user" -> JsString(obj.user) - ) + def writeExecutorRunner(obj: ExecutorRunner) = { + ("id" -> obj.execId) ~ + ("memory" -> obj.memory) ~ + ("appid" -> obj.appId) ~ + ("appdesc" -> writeApplicationDescription(obj.appDesc)) } - implicit object ExecutorRunnerJsonFormat extends RootJsonWriter[ExecutorRunner] { - def write(obj: ExecutorRunner) = JsObject( - "id" -> JsNumber(obj.execId), - "memory" -> JsNumber(obj.memory), - "appid" -> JsString(obj.appId), - "appdesc" -> obj.appDesc.toJson.asJsObject - ) + def writeMasterState(obj: MasterState) = { + ("url" -> ("spark://" + obj.uri)) ~ + ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ + ("cores" -> obj.workers.map(_.cores).sum) ~ + ("coresused" -> obj.workers.map(_.coresUsed).sum) ~ + ("memory" -> obj.workers.map(_.memory).sum) ~ + ("memoryused" -> obj.workers.map(_.memoryUsed).sum) ~ + ("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~ + ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) } - implicit object MasterStateJsonFormat extends RootJsonWriter[MasterState] { - def write(obj: MasterState) = JsObject( - "url" -> JsString("spark://" + obj.uri), - "workers" -> JsArray(obj.workers.toList.map(_.toJson)), - "cores" -> JsNumber(obj.workers.map(_.cores).sum), - "coresused" -> JsNumber(obj.workers.map(_.coresUsed).sum), - "memory" -> JsNumber(obj.workers.map(_.memory).sum), - "memoryused" -> JsNumber(obj.workers.map(_.memoryUsed).sum), - "activeapps" -> JsArray(obj.activeApps.toList.map(_.toJson)), - "completedapps" -> JsArray(obj.completedApps.toList.map(_.toJson)) - ) + def writeWorkerState(obj: WorkerState) = { + ("id" -> obj.workerId) ~ + ("masterurl" -> obj.masterUrl) ~ + ("masterwebuiurl" -> obj.masterWebUiUrl) ~ + ("cores" -> obj.cores) ~ + ("coresused" -> obj.coresUsed) ~ + ("memory" -> obj.memory) ~ + ("memoryused" -> obj.memoryUsed) ~ + ("executors" -> obj.executors.toList.map(writeExecutorRunner)) ~ + ("finishedexecutors" -> obj.finishedExecutors.toList.map(writeExecutorRunner)) } - - implicit object WorkerStateJsonFormat extends RootJsonWriter[WorkerState] { - def write(obj: WorkerState) = JsObject( - "id" -> JsString(obj.workerId), - "masterurl" -> JsString(obj.masterUrl), - "masterwebuiurl" -> JsString(obj.masterWebUiUrl), - "cores" -> JsNumber(obj.cores), - "coresused" -> JsNumber(obj.coresUsed), - "memory" -> JsNumber(obj.memory), - "memoryused" -> JsNumber(obj.memoryUsed), - "executors" -> JsArray(obj.executors.toList.map(_.toJson)), - "finishedexecutors" -> JsArray(obj.finishedExecutors.toList.map(_.toJson)) - ) - } -} +} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index a2e9dfd762..6623142d69 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -55,9 +55,9 @@ class MasterWebUI(master: ActorRef) extends Logging {
      -
    • ID: app.id
    • -
    • Description: app.desc.name
    • -
    • User: app.desc.user
    • +
    • ID: {app.id}
    • +
    • Description: {app.desc.name}
    • +
    • User: {app.desc.user}
    • Cores: { if (app.desc.maxCores == Integer.MAX_VALUE) { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index b8b4b89738..0af9eb8efa 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -1,16 +1,12 @@ package spark.deploy.worker -import akka.actor.{ActorRef, ActorSystem} +import akka.actor.ActorRef import akka.dispatch.Await import akka.pattern.ask import akka.util.{Duration, Timeout} import akka.util.duration._ -import cc.spray.Directives -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.http.MediaTypes import spark.deploy.{WorkerState, RequestWorkerState} -import spark.deploy.JsonProtocol._ import java.io.File import spark.util.{WebUI => UtilsWebUI} import spark.{Utils, Logging} diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 6ac4398de7..e9c362fce7 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -3,18 +3,12 @@ package spark.storage import akka.actor.{ActorRef, ActorSystem} import akka.util.Duration import akka.util.duration._ -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.Directives import spark.{Logging, SparkContext} -import spark.util.AkkaUtils import spark.Utils import spark.util.WebUI -import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler, ResourceHandler} import org.eclipse.jetty.server.Handler -import javax.servlet.http.{HttpServletResponse, HttpServletRequest} -import xml.Elem +import javax.servlet.http.HttpServletRequest import xml.Node -import java.net.URLClassLoader import spark.util.WebUI._ @@ -23,7 +17,7 @@ import spark.util.WebUI._ */ private[spark] class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, sc: SparkContext) - extends Directives with Logging { + extends Logging { implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() @@ -55,7 +49,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, val filteredStorageStatusList = StorageUtils. filterStorageStatusByPrefix(storageStatusList, prefix) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) val content =
      diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index bd2d637ae7..134c912c46 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,21 +1,10 @@ package spark.util -import akka.actor.{ActorRef, Props, ActorSystemImpl, ActorSystem} +import akka.actor.{ActorSystemImpl, ActorSystem} import com.typesafe.config.ConfigFactory import akka.util.duration._ -import akka.pattern.ask import akka.remote.RemoteActorRefProvider -import cc.spray.Route -import cc.spray.io.IoWorker -import cc.spray.{SprayCanRootService, HttpService} -import cc.spray.can.server.HttpServer -import cc.spray.io.pipelines.MessageHandlerDispatch.SingletonHandler -import akka.dispatch.Await -import spark.{Utils, SparkException} -import java.util.concurrent.TimeoutException -import org.eclipse.jetty.server.Server -import org.eclipse.jetty.server.Handler -import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler} + /** * Various utility classes for working with Akka. diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index 34b776f1d8..e6b39b15eb 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -1,23 +1,22 @@ package spark.util -import xml.Elem import xml.Node -import util.parsing.json.{JSONFormat, JSONObject} import org.eclipse.jetty.server.{Server, Request, Handler} import javax.servlet.http.{HttpServletResponse, HttpServletRequest} -import org.eclipse.jetty.util.component.LifeCycle.Listener import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} import util.Try import util.Success import util.Failure import spark.Logging import annotation.tailrec +import net.liftweb.json.JsonAST.JValue +import net.liftweb.json._ object WebUI extends Logging { type Responder[T] = HttpServletRequest => T - implicit def jsonResponderToHandler(responder: Responder[JSONObject]): Handler = - createHandler(responder, "text/json") + implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = + createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = createHandler(responder, "text/html") @@ -25,7 +24,8 @@ object WebUI extends Logging { implicit def textResponderToHandler(responder: Responder[String]): Handler = createHandler(responder, "text/plain") - def createHandler[T <% AnyRef](responder: Responder[T], contentType: String): Handler = { + def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, + extractFn: T => String = (in: Any) => in.toString): Handler = { new AbstractHandler { def handle(target: String, baseRequest: Request, diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index faf6e2ae8e..ec26b2a229 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -4,7 +4,6 @@ import sbt.Classpaths.publishTask import Keys._ import sbtassembly.Plugin._ import AssemblyKeys._ -import twirl.sbt.TwirlPlugin._ // For Sonatype publishing //import com.jsuereth.pgp.sbtplugin.PgpKeys._ @@ -157,9 +156,7 @@ object SparkBuild extends Build { "com.typesafe.akka" % "akka-slf4j" % "2.0.3" excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", - "cc.spray" % "spray-can" % "1.0-M2.1" excludeAll(excludeNetty), - "cc.spray" % "spray-server" % "1.0-M2.1" excludeAll(excludeNetty), - "cc.spray" % "spray-json_2.9.2" % "1.1.1" excludeAll(excludeNetty), + "net.liftweb" % "lift-json_2.9.2" % "2.5", "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test" @@ -189,7 +186,7 @@ object SparkBuild extends Build { "src/hadoop" + HADOOP_MAJOR_VERSION + "/scala" } ) } - ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings + ) ++ assemblySettings ++ extraAssemblySettings def rootSettings = sharedSettings ++ Seq( publish := {} diff --git a/project/plugins.sbt b/project/plugins.sbt index f806e66481..1b0f879b94 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -10,8 +10,6 @@ addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") -addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") - // For Sonatype publishing //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) From d6fde4ffe402b406ac9d9ea6d5c3ee0fb682f628 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 15:34:54 -0700 Subject: [PATCH 019/419] Some JSON cleanup --- .../scala/spark/deploy/master/MasterWebUI.scala | 13 +++++++++++++ .../scala/spark/deploy/worker/WorkerWebUI.scala | 11 +++++++++-- core/src/main/scala/spark/util/WebUI.scala | 5 ++--- 3 files changed, 24 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 6623142d69..49300ea742 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -14,6 +14,7 @@ import org.eclipse.jetty.server.Handler import spark.util.WebUI._ import spark.deploy.MasterState import javax.servlet.http.HttpServletRequest +import net.liftweb.json.JsonAST.JValue /** * Web UI server for the standalone master. @@ -39,10 +40,22 @@ class MasterWebUI(master: ActorRef) extends Logging { val handlers = Array[(String, Handler)]( ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), + ("/app/json", (request: HttpServletRequest) => appDetailJson(request)), ("/app", (request: HttpServletRequest) => appDetail(request)), ("*", (request: HttpServletRequest) => index) ) + /** Executor details for a particular application */ + def appDetailJson(request: HttpServletRequest): JValue = { + val appId = request.getParameter("appId") + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + val app = state.activeApps.find(_.id == appId).getOrElse({ + state.completedApps.find(_.id == appId).getOrElse(null) + }) + JsonProtocol.writeApplicationInfo(app) + } + /** Executor details for a particular application */ def appDetail(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 0af9eb8efa..3257f0f513 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -6,16 +6,16 @@ import akka.pattern.ask import akka.util.{Duration, Timeout} import akka.util.duration._ -import spark.deploy.{WorkerState, RequestWorkerState} +import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} import java.io.File import spark.util.{WebUI => UtilsWebUI} import spark.{Utils, Logging} import org.eclipse.jetty.server.Handler import spark.util.WebUI._ -import spark.deploy.WorkerState import scala.io.Source import javax.servlet.http.HttpServletRequest import xml.Node +import net.liftweb.json.JsonAST.JValue /** * Web UI server for the standalone worker. @@ -31,6 +31,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { val handlers = Array[(String, Handler)]( ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), ("/log", (request: HttpServletRequest) => log(request)), + ("/json", (request: HttpServletRequest) => indexJson), ("*", (request: HttpServletRequest) => index) ) @@ -45,6 +46,12 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { } } + def indexJson(): JValue = { + val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val workerState = Await.result(stateFuture, 3 seconds) + JsonProtocol.writeWorkerState(workerState) + } + def index(): Seq[Node] = { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 3 seconds) diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index e6b39b15eb..4d9df55a4f 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -34,9 +34,8 @@ object WebUI extends Logging { response.setContentType("%s;charset=utf-8".format(contentType)) response.setStatus(HttpServletResponse.SC_OK) baseRequest.setHandled(true) - response.getWriter().println( - responder(request).toString - ) + val result = responder(request) + response.getWriter().println(extractFn(result)) } } } From 4e1f202481844d950ef2a80679bdc9cee85f0b75 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 15:37:27 -0700 Subject: [PATCH 020/419] Removing dead code --- core/src/main/scala/spark/util/WebUI.scala | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index 4d9df55a4f..ae69c159e7 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -40,7 +40,6 @@ object WebUI extends Logging { } } - /** Create and return a staticHandler if resourceBase can be located */ def createStaticHandler(resourceBase: String): ResourceHandler = { val staticHandler = new ResourceHandler Option(getClass.getClassLoader.getResource(resourceBase)) match { @@ -50,19 +49,6 @@ object WebUI extends Logging { } } - /* - /** Create and return a staticHandler if resourceBase can be located */ - def createStaticHandler(resourceBase: String): Option[ResourceHandler] = { - val staticHandler = new ResourceHandler - Option(getClass.getClassLoader.getResource(resourceBase)) match { - case Some(res) => - staticHandler.setResourceBase (res.toString) - Some(staticHandler) - case None => None - } - } - */ - def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]): (Server, Int) = { val handlersToRegister = handlers.map { case(path, handler) => if (path == "*") { From 17f145f3bcdee12d2d965af3bc2ae3f9aa2d8ff3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 16:50:23 -0700 Subject: [PATCH 021/419] Updating Maven build --- core/pom.xml | 24 ++++-------------------- pom.xml | 51 +++++---------------------------------------------- 2 files changed, 9 insertions(+), 66 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 88f0ed70f3..5edafb3706 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -55,6 +55,10 @@ com.typesafe.akka akka-slf4j + + net.liftweb + lift-json_2.9.2 + it.unimi.dsi fastutil @@ -63,22 +67,6 @@ colt colt - - cc.spray - spray-can - - - cc.spray - spray-server - - - cc.spray - spray-json_2.9.2 - - - org.tomdz.twirl - twirl-api - com.github.scala-incubator.io scala-io-file_2.9.2 @@ -171,10 +159,6 @@ - - org.tomdz.twirl - twirl-maven-plugin - diff --git a/pom.xml b/pom.xml index 3bcb2a3f34..81043e3b6c 100644 --- a/pom.xml +++ b/pom.xml @@ -54,8 +54,6 @@ 2.9.3 0.9.0-incubating 2.0.3 - 1.0-M2.1 - 1.1.1 1.6.1 4.1.2 1.2.17 @@ -98,17 +96,6 @@ false - - spray-repo - Spray Repository - http://repo.spray.cc/ - - true - - - false - - twitter4j-repo Twitter4J Repository @@ -229,26 +216,6 @@ colt 1.2.0 - - cc.spray - spray-can - ${spray.version} - - - cc.spray - spray-server - ${spray.version} - - - cc.spray - spray-json_2.9.2 - ${spray.json.version} - - - org.tomdz.twirl - twirl-api - 1.0.2 - com.github.scala-incubator.io scala-io-file_2.9.2 @@ -270,7 +237,11 @@ 10.4.2.0 test - + + net.liftweb + lift-json_2.9.2 + 2.5 + org.scala-lang scala-compiler @@ -346,18 +317,6 @@ build-helper-maven-plugin 1.7 - - org.tomdz.twirl - twirl-maven-plugin - 1.0.1 - - - - generate - - - - net.alchim31.maven scala-maven-plugin From 32a45d01b104f79d9e5f10c333c44cbf910949ef Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 18:45:14 -0700 Subject: [PATCH 022/419] Removing twirl files --- core/src/main/scala/spark/util/WebUI.scala | 12 ++- .../main/twirl/spark/common/layout.scala.html | 35 -------- .../deploy/master/app_details.scala.html | 38 --------- .../spark/deploy/master/app_row.scala.html | 20 ----- .../spark/deploy/master/app_table.scala.html | 21 ----- .../deploy/master/executor_row.scala.html | 15 ---- .../deploy/master/executors_table.scala.html | 19 ----- .../spark/deploy/master/index.scala.html | 53 ------------ .../spark/deploy/master/worker_row.scala.html | 14 ---- .../deploy/master/worker_table.scala.html | 18 ----- .../deploy/worker/executor_row.scala.html | 20 ----- .../deploy/worker/executors_table.scala.html | 18 ----- .../spark/deploy/worker/index.scala.html | 44 ---------- .../main/twirl/spark/storage/index.scala.html | 40 --------- .../main/twirl/spark/storage/rdd.scala.html | 81 ------------------- .../twirl/spark/storage/rdd_table.scala.html | 32 -------- .../spark/storage/worker_table.scala.html | 24 ------ 17 files changed, 8 insertions(+), 496 deletions(-) delete mode 100644 core/src/main/twirl/spark/common/layout.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/app_details.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/app_row.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/app_table.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/executor_row.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/executors_table.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/index.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/worker_row.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/worker_table.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/worker/executor_row.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/worker/executors_table.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/worker/index.scala.html delete mode 100644 core/src/main/twirl/spark/storage/index.scala.html delete mode 100644 core/src/main/twirl/spark/storage/rdd.scala.html delete mode 100644 core/src/main/twirl/spark/storage/rdd_table.scala.html delete mode 100644 core/src/main/twirl/spark/storage/worker_table.scala.html diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index ae69c159e7..b9ddfdb40f 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -86,6 +86,9 @@ object WebUI extends Logging { {title} +
      @@ -98,9 +101,10 @@ object WebUI extends Logging {
      - {content} -
    - - +
    + {content} +
    + + } } diff --git a/core/src/main/twirl/spark/common/layout.scala.html b/core/src/main/twirl/spark/common/layout.scala.html deleted file mode 100644 index b9192060aa..0000000000 --- a/core/src/main/twirl/spark/common/layout.scala.html +++ /dev/null @@ -1,35 +0,0 @@ -@(title: String)(content: Html) - - - - - - - - - - @title - - - - -
    - - -
    -
    - -

    @title

    -
    -
    - -
    - - @content - -
    - - - \ No newline at end of file diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html deleted file mode 100644 index 5e5e5de551..0000000000 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ /dev/null @@ -1,38 +0,0 @@ -@(app: spark.deploy.master.ApplicationInfo) - -@spark.common.html.layout(title = "Application Details") { - - -
    -
    -
      -
    • ID: @app.id
    • -
    • Description: @app.desc.name
    • -
    • User: @app.desc.user
    • -
    • Cores: - @if(app.desc.maxCores == Integer.MAX_VALUE) { - Unlimited (@app.coresGranted granted) - } else { - @app.desc.maxCores (@app.coresGranted granted, @app.coresLeft left) - } -
    • -
    • Memory per Slave: @app.desc.memoryPerSlave
    • -
    • Submit Date: @app.submitDate
    • -
    • State: @app.state
    • -
    • Application Detail UI
    • -
    -
    -
    - -
    - - -
    -
    -

    Executor Summary

    -
    - @executors_table(app.executors.values.toList) -
    -
    - -} diff --git a/core/src/main/twirl/spark/deploy/master/app_row.scala.html b/core/src/main/twirl/spark/deploy/master/app_row.scala.html deleted file mode 100644 index feb306f35c..0000000000 --- a/core/src/main/twirl/spark/deploy/master/app_row.scala.html +++ /dev/null @@ -1,20 +0,0 @@ -@(app: spark.deploy.master.ApplicationInfo) - -@import spark.Utils -@import spark.deploy.WebUI.formatDate -@import spark.deploy.WebUI.formatDuration - - - - @app.id - - @app.desc.name - - @app.coresGranted - - @Utils.memoryMegabytesToString(app.desc.memoryPerSlave) - @formatDate(app.submitDate) - @app.desc.user - @app.state.toString() - @formatDuration(app.duration) - diff --git a/core/src/main/twirl/spark/deploy/master/app_table.scala.html b/core/src/main/twirl/spark/deploy/master/app_table.scala.html deleted file mode 100644 index f789cee0f1..0000000000 --- a/core/src/main/twirl/spark/deploy/master/app_table.scala.html +++ /dev/null @@ -1,21 +0,0 @@ -@(apps: Array[spark.deploy.master.ApplicationInfo]) - - - - - - - - - - - - - - - - @for(j <- apps) { - @app_row(j) - } - -
    IDDescriptionCoresMemory per NodeSubmit TimeUserStateDuration
    diff --git a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html deleted file mode 100644 index 21e72c7aab..0000000000 --- a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html +++ /dev/null @@ -1,15 +0,0 @@ -@(executor: spark.deploy.master.ExecutorInfo) - - - @executor.id - - @executor.worker.id - - @executor.cores - @executor.memory - @executor.state - - stdout - stderr - - diff --git a/core/src/main/twirl/spark/deploy/master/executors_table.scala.html b/core/src/main/twirl/spark/deploy/master/executors_table.scala.html deleted file mode 100644 index cafc42c80e..0000000000 --- a/core/src/main/twirl/spark/deploy/master/executors_table.scala.html +++ /dev/null @@ -1,19 +0,0 @@ -@(executors: List[spark.deploy.master.ExecutorInfo]) - - - - - - - - - - - - - - @for(e <- executors) { - @executor_row(e) - } - -
    ExecutorIDWorkerCoresMemoryStateLogs
    \ No newline at end of file diff --git a/core/src/main/twirl/spark/deploy/master/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html deleted file mode 100644 index b9b9f08810..0000000000 --- a/core/src/main/twirl/spark/deploy/master/index.scala.html +++ /dev/null @@ -1,53 +0,0 @@ -@(state: spark.deploy.MasterState) -@import spark.deploy.master._ -@import spark.Utils - -@spark.common.html.layout(title = "Spark Master on " + state.host + ":" + state.port) { - - -
    -
    -
      -
    • URL: @(state.uri)
    • -
    • Workers: @state.workers.size
    • -
    • Cores: @{state.workers.map(_.cores).sum} Total, - @{state.workers.map(_.coresUsed).sum} Used
    • -
    • Memory: @{Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, - @{Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
    • -
    • Applications: @state.activeApps.size Running, @state.completedApps.size Completed
    • -
    -
    -
    - - -
    -
    -

    Workers

    -
    - @worker_table(state.workers.sortBy(_.id)) -
    -
    - -
    - - -
    -
    -

    Running Applications

    -
    - @app_table(state.activeApps.sortBy(_.startTime).reverse) -
    -
    - -
    - - -
    -
    -

    Completed Applications

    -
    - @app_table(state.completedApps.sortBy(_.endTime).reverse) -
    -
    - -} diff --git a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html deleted file mode 100644 index 46277ca421..0000000000 --- a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html +++ /dev/null @@ -1,14 +0,0 @@ -@(worker: spark.deploy.master.WorkerInfo) - -@import spark.Utils - - - - @worker.id - - @{worker.host}:@{worker.port} - @worker.state - @worker.cores (@worker.coresUsed Used) - @{Utils.memoryMegabytesToString(worker.memory)} - (@{Utils.memoryMegabytesToString(worker.memoryUsed)} Used) - diff --git a/core/src/main/twirl/spark/deploy/master/worker_table.scala.html b/core/src/main/twirl/spark/deploy/master/worker_table.scala.html deleted file mode 100644 index b249411a62..0000000000 --- a/core/src/main/twirl/spark/deploy/master/worker_table.scala.html +++ /dev/null @@ -1,18 +0,0 @@ -@(workers: Array[spark.deploy.master.WorkerInfo]) - - - - - - - - - - - - - @for(w <- workers) { - @worker_row(w) - } - -
    IDAddressStateCoresMemory
    diff --git a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html deleted file mode 100644 index dad0a89080..0000000000 --- a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html +++ /dev/null @@ -1,20 +0,0 @@ -@(executor: spark.deploy.worker.ExecutorRunner) - -@import spark.Utils - - - @executor.execId - @executor.cores - @Utils.memoryMegabytesToString(executor.memory) - -
      -
    • ID: @executor.appId
    • -
    • Name: @executor.appDesc.name
    • -
    • User: @executor.appDesc.user
    • -
    - - - stdout - stderr - - diff --git a/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html b/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html deleted file mode 100644 index 327a2399c7..0000000000 --- a/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html +++ /dev/null @@ -1,18 +0,0 @@ -@(executors: List[spark.deploy.worker.ExecutorRunner]) - - - - - - - - - - - - - @for(e <- executors) { - @executor_row(e) - } - -
    ExecutorIDCoresMemoryJob DetailsLogs
    \ No newline at end of file diff --git a/core/src/main/twirl/spark/deploy/worker/index.scala.html b/core/src/main/twirl/spark/deploy/worker/index.scala.html deleted file mode 100644 index 0e66af9284..0000000000 --- a/core/src/main/twirl/spark/deploy/worker/index.scala.html +++ /dev/null @@ -1,44 +0,0 @@ -@(worker: spark.deploy.WorkerState) -@import spark.Utils - -@spark.common.html.layout(title = "Spark Worker on " + worker.host + ":" + worker.port) { - - -
    -
    -
      -
    • ID: @worker.workerId
    • -
    • - Master URL: @worker.masterUrl -
    • -
    • Cores: @worker.cores (@worker.coresUsed Used)
    • -
    • Memory: @{Utils.memoryMegabytesToString(worker.memory)} - (@{Utils.memoryMegabytesToString(worker.memoryUsed)} Used)
    • -
    -

    Back to Master

    -
    -
    - -
    - - -
    -
    -

    Running Executors

    -
    - @executors_table(worker.executors) -
    -
    - -
    - - -
    -
    -

    Finished Executors

    -
    - @executors_table(worker.finishedExecutors) -
    -
    - -} diff --git a/core/src/main/twirl/spark/storage/index.scala.html b/core/src/main/twirl/spark/storage/index.scala.html deleted file mode 100644 index 2b337f6133..0000000000 --- a/core/src/main/twirl/spark/storage/index.scala.html +++ /dev/null @@ -1,40 +0,0 @@ -@(maxMem: Long, remainingMem: Long, diskSpaceUsed: Long, rdds: Array[spark.storage.RDDInfo], storageStatusList: Array[spark.storage.StorageStatus]) -@import spark.Utils - -@spark.common.html.layout(title = "Storage Dashboard") { - - -
    -
    -
      -
    • Memory: - @{Utils.memoryBytesToString(maxMem - remainingMem)} Used - (@{Utils.memoryBytesToString(remainingMem)} Available)
    • -
    • Disk: @{Utils.memoryBytesToString(diskSpaceUsed)} Used
    • -
    -
    -
    - -
    - - -
    -
    -

    RDD Summary

    -
    - @rdd_table(rdds) -
    -
    - -
    - - -
    -
    -

    Worker Summary

    -
    - @worker_table(storageStatusList) -
    -
    - -} \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd.scala.html b/core/src/main/twirl/spark/storage/rdd.scala.html deleted file mode 100644 index d85addeb17..0000000000 --- a/core/src/main/twirl/spark/storage/rdd.scala.html +++ /dev/null @@ -1,81 +0,0 @@ -@(rddInfo: spark.storage.RDDInfo, storageStatusList: Array[spark.storage.StorageStatus]) -@import spark.Utils - -@spark.common.html.layout(title = "RDD Info ") { - - -
    -
    -
      -
    • - Storage Level: - @(rddInfo.storageLevel.description) -
    • - Cached Partitions: - @(rddInfo.numCachedPartitions) -
    • -
    • - Total Partitions: - @(rddInfo.numPartitions) -
    • -
    • - Memory Size: - @{Utils.memoryBytesToString(rddInfo.memSize)} -
    • -
    • - Disk Size: - @{Utils.memoryBytesToString(rddInfo.diskSize)} -
    • -
    -
    -
    - -
    - - -
    -
    -

    RDD Summary

    -
    - - - - - - - - - - - - - - @storageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { case (k,v) => - - - - - - - } - -
    Block NameStorage LevelSize in MemorySize on Disk
    @k - @(v.storageLevel.description) - @{Utils.memoryBytesToString(v.memSize)}@{Utils.memoryBytesToString(v.diskSize)}
    - - -
    -
    - -
    - - -
    -
    -

    Worker Summary

    -
    - @worker_table(storageStatusList, "rdd_" + rddInfo.id ) -
    -
    - -} \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd_table.scala.html b/core/src/main/twirl/spark/storage/rdd_table.scala.html deleted file mode 100644 index a51e64aed0..0000000000 --- a/core/src/main/twirl/spark/storage/rdd_table.scala.html +++ /dev/null @@ -1,32 +0,0 @@ -@(rdds: Array[spark.storage.RDDInfo]) -@import spark.Utils - - - - - - - - - - - - - - @for(rdd <- rdds) { - - - - - - - - - } - -
    RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
    - - @rdd.name - - @(rdd.storageLevel.description) - @rdd.numCachedPartitions@(rdd.numCachedPartitions / rdd.numPartitions.toDouble)@{Utils.memoryBytesToString(rdd.memSize)}@{Utils.memoryBytesToString(rdd.diskSize)}
    \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/worker_table.scala.html b/core/src/main/twirl/spark/storage/worker_table.scala.html deleted file mode 100644 index cd72a688c1..0000000000 --- a/core/src/main/twirl/spark/storage/worker_table.scala.html +++ /dev/null @@ -1,24 +0,0 @@ -@(workersStatusList: Array[spark.storage.StorageStatus], prefix: String = "") -@import spark.Utils - - - - - - - - - - - @for(status <- workersStatusList) { - - - - - - } - -
    HostMemory UsageDisk Usage
    @(status.blockManagerId.host + ":" + status.blockManagerId.port) - @(Utils.memoryBytesToString(status.memUsed(prefix))) - (@(Utils.memoryBytesToString(status.memRemaining)) Total Available) - @(Utils.memoryBytesToString(status.diskUsed(prefix)))
    \ No newline at end of file From 8b5c7e71c4cbf8b8ad9ab9c81f3b21ee3da49e24 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 19:02:20 -0700 Subject: [PATCH 023/419] Import cleanup --- core/src/main/scala/spark/deploy/WebUI.scala | 2 +- .../spark/deploy/master/MasterWebUI.scala | 29 +++++++++---------- .../spark/deploy/worker/WorkerWebUI.scala | 15 +++++----- .../scala/spark/storage/BlockManagerUI.scala | 10 +++---- core/src/main/scala/spark/util/WebUI.scala | 15 ++++------ 5 files changed, 32 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/spark/deploy/WebUI.scala b/core/src/main/scala/spark/deploy/WebUI.scala index ad1a1092b2..844c4142c7 100644 --- a/core/src/main/scala/spark/deploy/WebUI.scala +++ b/core/src/main/scala/spark/deploy/WebUI.scala @@ -6,7 +6,7 @@ import java.util.Date /** * Utilities used throughout the web UI. */ -private[spark] object WebUI { +private[spark] object DeployWebUI { val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") def formatDate(date: Date): String = DATE_FORMAT.format(date) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 49300ea742..171e74a3a6 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -1,20 +1,19 @@ package spark.deploy.master -import akka.actor.{ActorRef} +import akka.actor.ActorRef import akka.dispatch.Await import akka.pattern.ask -import akka.util.{Duration} +import akka.util.Duration import akka.util.duration._ -import scala.xml.Node -import spark.{Logging, Utils} -import spark.util.{WebUI => UtilsWebUI} - -import spark.deploy._ -import org.eclipse.jetty.server.Handler -import spark.util.WebUI._ -import spark.deploy.MasterState import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue +import org.eclipse.jetty.server.Handler +import scala.xml.Node +import spark.{Logging, Utils} +import spark.util.WebUI +import spark.util.WebUI._ +import spark.deploy._ +import spark.deploy.MasterState /** * Web UI server for the standalone master. @@ -29,7 +28,7 @@ class MasterWebUI(master: ActorRef) extends Logging { def start() { try { - val (server, boundPort) = UtilsWebUI.startJettyServer("0.0.0.0", port, handlers) + val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, handlers) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => @@ -98,7 +97,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {executorTable(app.executors.values.toList)} ; - UtilsWebUI.makePage(content, "Application Info: " + app.desc.name) + WebUI.makePage(content, "Application Info: " + app.desc.name) } def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -187,7 +186,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {appTable(state.completedApps.sortBy(_.endTime).reverse)} ; - UtilsWebUI.makePage(content, "Spark Master: " + state.uri) + WebUI.makePage(content, "Spark Master: " + state.uri) } def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { @@ -246,10 +245,10 @@ class MasterWebUI(master: ActorRef) extends Logging { {app.coresGranted} {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} - {WebUI.formatDate(app.submitDate)} + {DeployWebUI.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} - {WebUI.formatDuration(app.duration)} + {DeployWebUI.formatDuration(app.duration)} } } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 3257f0f513..80b285b3f6 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -5,17 +5,16 @@ import akka.dispatch.Await import akka.pattern.ask import akka.util.{Duration, Timeout} import akka.util.duration._ - -import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} import java.io.File -import spark.util.{WebUI => UtilsWebUI} -import spark.{Utils, Logging} -import org.eclipse.jetty.server.Handler -import spark.util.WebUI._ -import scala.io.Source import javax.servlet.http.HttpServletRequest -import xml.Node import net.liftweb.json.JsonAST.JValue +import org.eclipse.jetty.server.Handler +import scala.io.Source +import spark.{Utils, Logging} +import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} +import spark.util.{WebUI => UtilsWebUI} +import spark.util.WebUI._ +import xml.Node /** * Web UI server for the standalone worker. diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index e9c362fce7..ecffd1edcf 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -2,15 +2,13 @@ package spark.storage import akka.actor.{ActorRef, ActorSystem} import akka.util.Duration -import akka.util.duration._ +import javax.servlet.http.HttpServletRequest +import org.eclipse.jetty.server.Handler import spark.{Logging, SparkContext} import spark.Utils -import spark.util.WebUI -import org.eclipse.jetty.server.Handler -import javax.servlet.http.HttpServletRequest -import xml.Node import spark.util.WebUI._ - +import spark.util.WebUI +import xml.Node /** * Web UI server for the BlockManager inside each SparkContext. diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index b9ddfdb40f..bde1acdbca 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -1,16 +1,13 @@ package spark.util -import xml.Node -import org.eclipse.jetty.server.{Server, Request, Handler} -import javax.servlet.http.{HttpServletResponse, HttpServletRequest} -import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} -import util.Try -import util.Success -import util.Failure -import spark.Logging import annotation.tailrec -import net.liftweb.json.JsonAST.JValue +import javax.servlet.http.{HttpServletResponse, HttpServletRequest} import net.liftweb.json._ +import org.eclipse.jetty.server.{Server, Request, Handler} +import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} +import spark.Logging +import util.{Try, Success, Failure} +import xml.Node object WebUI extends Logging { type Responder[T] = HttpServletRequest => T From 77c53f7868a036dc7712865e3a43d9fdea47cc98 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 22:18:04 -0700 Subject: [PATCH 024/419] Refactoring UI packages --- .../static/bootstrap-responsive.min.css | 0 .../static/bootstrap.min.css | 0 .../{deploy => webui}/static/sorttable.js | 0 .../{deploy => webui}/static/spark_logo.png | Bin core/src/main/scala/spark/SparkContext.scala | 8 +-- .../spark/deploy/master/MasterWebUI.scala | 13 +++-- .../spark/deploy/worker/WorkerWebUI.scala | 42 ++-------------- .../{storage => ui}/BlockManagerUI.scala | 47 +++++------------- core/src/main/scala/spark/ui/SparkUI.scala | 41 +++++++++++++++ .../main/scala/spark/{util => ui}/WebUI.scala | 30 ++++++++--- 10 files changed, 93 insertions(+), 88 deletions(-) rename core/src/main/resources/spark/{deploy => webui}/static/bootstrap-responsive.min.css (100%) rename core/src/main/resources/spark/{deploy => webui}/static/bootstrap.min.css (100%) rename core/src/main/resources/spark/{deploy => webui}/static/sorttable.js (100%) rename core/src/main/resources/spark/{deploy => webui}/static/spark_logo.png (100%) rename core/src/main/scala/spark/{storage => ui}/BlockManagerUI.scala (79%) create mode 100644 core/src/main/scala/spark/ui/SparkUI.scala rename core/src/main/scala/spark/{util => ui}/WebUI.scala (79%) diff --git a/core/src/main/resources/spark/deploy/static/bootstrap-responsive.min.css b/core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css similarity index 100% rename from core/src/main/resources/spark/deploy/static/bootstrap-responsive.min.css rename to core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css diff --git a/core/src/main/resources/spark/deploy/static/bootstrap.min.css b/core/src/main/resources/spark/webui/static/bootstrap.min.css similarity index 100% rename from core/src/main/resources/spark/deploy/static/bootstrap.min.css rename to core/src/main/resources/spark/webui/static/bootstrap.min.css diff --git a/core/src/main/resources/spark/deploy/static/sorttable.js b/core/src/main/resources/spark/webui/static/sorttable.js similarity index 100% rename from core/src/main/resources/spark/deploy/static/sorttable.js rename to core/src/main/resources/spark/webui/static/sorttable.js diff --git a/core/src/main/resources/spark/deploy/static/spark_logo.png b/core/src/main/resources/spark/webui/static/spark_logo.png similarity index 100% rename from core/src/main/resources/spark/deploy/static/spark_logo.png rename to core/src/main/resources/spark/webui/static/spark_logo.png diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 70a9d7698c..cc634b3ec4 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -46,8 +46,9 @@ import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo} +import spark.storage.{StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} +import ui.{SparkUI, BlockManagerUI} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -93,9 +94,8 @@ class SparkContext( isLocal) SparkEnv.set(env) - // Start the BlockManager UI - private[spark] val ui = new BlockManagerUI( - env.actorSystem, env.blockManager.master.driverActor, this) + // Start the Spark UI + private[spark] val ui = new SparkUI(this) ui.start() // Used to store a URL for each static file/jar together with the file's local timestamp diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 171e74a3a6..328a7cb297 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -10,10 +10,11 @@ import net.liftweb.json.JsonAST.JValue import org.eclipse.jetty.server.Handler import scala.xml.Node import spark.{Logging, Utils} -import spark.util.WebUI -import spark.util.WebUI._ +import spark.ui.WebUI +import WebUI._ import spark.deploy._ import spark.deploy.MasterState +import spark.ui.WebUI /** * Web UI server for the standalone master. @@ -64,6 +65,7 @@ class MasterWebUI(master: ActorRef) extends Logging { state.completedApps.find(_.id == appId).getOrElse(null) }) val content = +
      @@ -97,7 +99,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {executorTable(app.executors.values.toList)}
    ; - WebUI.makePage(content, "Application Info: " + app.desc.name) + WebUI.sparkPage(content, "Application Info: " + app.desc.name) } def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -142,6 +144,7 @@ class MasterWebUI(master: ActorRef) extends Logging { val state = Await.result(stateFuture, 3 seconds) val content = +
      @@ -186,7 +189,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {appTable(state.completedApps.sortBy(_.endTime).reverse)}
    ; - WebUI.makePage(content, "Spark Master: " + state.uri) + WebUI.sparkPage(content, "Spark Master: " + state.uri) } def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { @@ -258,6 +261,6 @@ class MasterWebUI(master: ActorRef) extends Logging { } object MasterWebUI { - val STATIC_RESOURCE_DIR = "spark/deploy/static" + val STATIC_RESOURCE_DIR = "spark/webui/static" val DEFAULT_PORT = "8080" } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 80b285b3f6..f661d99815 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -12,8 +12,8 @@ import org.eclipse.jetty.server.Handler import scala.io.Source import spark.{Utils, Logging} import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} -import spark.util.{WebUI => UtilsWebUI} -import spark.util.WebUI._ +import spark.ui.{WebUI => UtilsWebUI} +import spark.ui.WebUI._ import xml.Node /** @@ -55,6 +55,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 3 seconds) val content = +
      @@ -88,7 +89,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging {
    ; - UtilsWebUI.makePage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + UtilsWebUI.sparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) } def executorTable(executors: Seq[ExecutorRunner]): Seq[Node] = { @@ -139,42 +140,9 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { source.close() lines } - - /* - val handler = { - get { - (path("") & parameters('format ?)) { - case Some(js) if js.equalsIgnoreCase("json") => { - val future = worker ? RequestWorkerState - respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(future.mapTo[WorkerState]) - } - } - case _ => - completeWith{ - val future = worker ? RequestWorkerState - future.map { workerState => - spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState]) - } - } - } ~ - path("log") { - parameters("appId", "executorId", "logType") { (appId, executorId, logType) => - respondWithMediaType(cc.spray.http.MediaTypes.`text/plain`) { - getFromFileName(workDir.getPath() + "/" + appId + "/" + executorId + "/" + logType) - } - } - } ~ - pathPrefix("static") { - getFromResourceDirectory(STATIC_RESOURCE_DIR) - } ~ - getFromResourceDirectory(RESOURCE_DIR) - } - } - */ } object WorkerWebUI { - val STATIC_RESOURCE_DIR = "spark/deploy/static" + val STATIC_RESOURCE_DIR = "spark/webui/static" val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/BlockManagerUI.scala similarity index 79% rename from core/src/main/scala/spark/storage/BlockManagerUI.scala rename to core/src/main/scala/spark/ui/BlockManagerUI.scala index ecffd1edcf..f319751590 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/BlockManagerUI.scala @@ -1,4 +1,4 @@ -package spark.storage +package spark.ui import akka.actor.{ActorRef, ActorSystem} import akka.util.Duration @@ -6,38 +6,23 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler import spark.{Logging, SparkContext} import spark.Utils -import spark.util.WebUI._ -import spark.util.WebUI +import WebUI._ import xml.Node +import spark.storage.StorageUtils /** * Web UI server for the BlockManager inside each SparkContext. */ private[spark] -class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, sc: SparkContext) - extends Logging { +class BlockManagerUI(sc: SparkContext) + extends UIComponent with Logging { + implicit val timeout = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - val host = Utils.localHostName() - val port = Option(System.getProperty("spark.ui.port")) - .getOrElse(BlockManagerUI.DEFAULT_PORT).toInt - /** Start a HTTP server to run the Web interface */ - def start() { - try { - val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, handlers) - logInfo("Started BlockManager web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create BlockManager WebUI", e) - System.exit(1) - } - } - - val handlers = Array[(String, Handler)]( - ("/static", createStaticHandler(BlockManagerUI.STATIC_RESOURCE_DIR)), - ("/rdd", (request: HttpServletRequest) => rddPage(request)), - ("*", (request: HttpServletRequest) => indexPage) + def getHandlers = Seq[(String, Handler)]( + ("/storage/rdd", (request: HttpServletRequest) => rddPage(request)), + ("/storage", (request: HttpServletRequest) => indexPage) ) def rddPage(request: HttpServletRequest): Seq[Node] = { @@ -137,7 +122,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, ; - WebUI.makePage(content, "RDD Info: " + id) + WebUI.headerSparkPage(content, "RDD Info: " + id) } def indexPage: Seq[Node] = { @@ -176,7 +161,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, {for (rdd <- rdds) yield - + {rdd.name} @@ -191,12 +176,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, ; - WebUI.makePage(content, "Spark Storage") + WebUI.headerSparkPage(content, "Spark Storage ") } - private[spark] def appUIAddress = "http://" + host + ":" + port -} - -object BlockManagerUI { - val STATIC_RESOURCE_DIR = "spark/deploy/static" - val DEFAULT_PORT = "33000" } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala new file mode 100644 index 0000000000..6c9affc1f7 --- /dev/null +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -0,0 +1,41 @@ +package spark.ui + +import spark.{Logging, SparkContext, Utils} +import javax.servlet.http.HttpServletRequest +import org.eclipse.jetty.server.Handler +import WebUI._ + +private[spark] class SparkUI(sc: SparkContext) extends Logging { + val host = Utils.localHostName() + val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt + + + val handlers = Seq[(String, Handler)]( + ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), + ("*", (request: HttpServletRequest) => WebUI.headerSparkPage(

    Test

    , "Test page")) + ) + val components = Seq(new BlockManagerUI(sc)) + + def start() { + /** Start an HTTP server to run the Web interface */ + try { + val allHandlers = components.flatMap(_.getHandlers) ++ handlers + val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, allHandlers) + logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Spark WebUI", e) + System.exit(1) + } + } + + private[spark] def appUIAddress = "http://" + host + ":" + port +} + +object SparkUI { + val DEFAULT_PORT = "33000" + val STATIC_RESOURCE_DIR = "spark/webui/static" +} + + + diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/ui/WebUI.scala similarity index 79% rename from core/src/main/scala/spark/util/WebUI.scala rename to core/src/main/scala/spark/ui/WebUI.scala index bde1acdbca..16251e727a 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/ui/WebUI.scala @@ -1,4 +1,4 @@ -package spark.util +package spark.ui import annotation.tailrec import javax.servlet.http.{HttpServletResponse, HttpServletRequest} @@ -9,6 +9,10 @@ import spark.Logging import util.{Try, Success, Failure} import xml.Node +abstract class UIComponent { + def getHandlers(): Seq[(String, Handler)] +} + object WebUI extends Logging { type Responder[T] = HttpServletRequest => T @@ -41,24 +45,27 @@ object WebUI extends Logging { val staticHandler = new ResourceHandler Option(getClass.getClassLoader.getResource(resourceBase)) match { case Some(res) => - staticHandler.setResourceBase (res.toString) - staticHandler + staticHandler.setResourceBase(res.toString) + case None => + logError("Could not find resource path for Web UI: " + resourceBase) } + staticHandler } - def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]): (Server, Int) = { + def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { val handlersToRegister = handlers.map { case(path, handler) => if (path == "*") { handler } else { val contextHandler = new ContextHandler(path) + println("Adding handler for path: " + path) contextHandler.setHandler(handler) contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] } } val handlerList = new HandlerList - handlerList.setHandlers(handlersToRegister) + handlerList.setHandlers(handlersToRegister.toArray) @tailrec def connect(currentPort: Int): (Server, Int) = { @@ -71,11 +78,19 @@ object WebUI extends Logging { connect((currentPort + 1) % 65536) } } - connect(port) } - def makePage(content: => Seq[Node], title: String): Seq[Node] = { + /** Page with Spark logo, title, and Spark UI headers */ + def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { + val newContent = +

    Storage | Jobs

    ; + + sparkPage(newContent ++ content, title) + } + + /** Page with Spark logo and title */ + def sparkPage(content: => Seq[Node], title: String): Seq[Node] = { @@ -98,7 +113,6 @@ object WebUI extends Logging { -
    {content} From bc4a811c57ee126559721a3a4284510baef184d1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 19 Jun 2013 12:56:01 -0700 Subject: [PATCH 025/419] Stash --- core/src/main/scala/spark/ui/JobProgressUI.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 core/src/main/scala/spark/ui/JobProgressUI.scala diff --git a/core/src/main/scala/spark/ui/JobProgressUI.scala b/core/src/main/scala/spark/ui/JobProgressUI.scala new file mode 100644 index 0000000000..eb782d85c8 --- /dev/null +++ b/core/src/main/scala/spark/ui/JobProgressUI.scala @@ -0,0 +1,14 @@ +package spark.ui + +import spark.SparkContext +import spark.scheduler.SparkListener + +private[spark] +class JobProgressUI(sc: SparkContext) { + sc.addSparkListener() +} + +class JobProgressListener extends SparkListener { + +} + From 9fd5dc3ea9016b59667fd903992843605087e43b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 09:02:56 -0700 Subject: [PATCH 026/419] Initial steps towards job progress UI --- core/src/main/scala/spark/SparkContext.scala | 8 +- .../scala/spark/scheduler/DAGScheduler.scala | 1 + .../main/scala/spark/scheduler/Stage.scala | 1 + .../main/scala/spark/ui/JobProgressUI.scala | 178 +++++++++++++++++- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- core/src/main/scala/spark/ui/WebUI.scala | 3 +- 6 files changed, 181 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index cc634b3ec4..155a5ee721 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -94,10 +94,6 @@ class SparkContext( isLocal) SparkEnv.set(env) - // Start the Spark UI - private[spark] val ui = new SparkUI(this) - ui.start() - // Used to store a URL for each static file/jar together with the file's local timestamp private[spark] val addedFiles = HashMap[String, Long]() private[spark] val addedJars = HashMap[String, Long]() @@ -215,6 +211,10 @@ class SparkContext( @volatile private var dagScheduler = new DAGScheduler(taskScheduler) dagScheduler.start() + // Start the Spark UI + private[spark] val ui = new SparkUI(this) + ui.start() + /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { val conf = SparkHadoopUtil.newConfiguration() diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f7d60be5db..bdd8792ce9 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -503,6 +503,7 @@ class DAGScheduler( case _ => "Unkown" } logInfo("%s (%s) finished in %s s".format(stage, stage.origin, serviceTime)) + stage.completionTime = Some(System.currentTimeMillis) val stageComp = StageCompleted(stageToInfos(stage)) sparkListeners.foreach{_.onStageCompleted(stageComp)} running -= stage diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 7fc9e13fd9..539cf8233b 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -34,6 +34,7 @@ private[spark] class Stage( /** When first task was submitted to scheduler. */ var submissionTime: Option[Long] = None + var completionTime: Option[Long] = None private var nextAttemptId = 0 diff --git a/core/src/main/scala/spark/ui/JobProgressUI.scala b/core/src/main/scala/spark/ui/JobProgressUI.scala index eb782d85c8..1f12df10b8 100644 --- a/core/src/main/scala/spark/ui/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/JobProgressUI.scala @@ -1,14 +1,182 @@ package spark.ui -import spark.SparkContext -import spark.scheduler.SparkListener +import spark.{Utils, SparkContext} +import spark.scheduler._ +import spark.scheduler.SparkListenerTaskEnd +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerStageSubmitted +import org.eclipse.jetty.server.Handler +import javax.servlet.http.HttpServletRequest +import xml.Node +import WebUI._ +import collection.mutable._ +import spark.Success +import akka.util.Duration +import java.text.SimpleDateFormat +import java.util.Date +import spark.scheduler.cluster.TaskInfo +import collection.mutable +import org.hsqldb.lib.HashMappedList +import spark.executor.TaskMetrics +import spark.scheduler.SparkListenerTaskEnd +import scala.Some +import spark.scheduler.SparkListenerStageSubmitted +import scala.Seq +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerJobStart private[spark] -class JobProgressUI(sc: SparkContext) { - sc.addSparkListener() +class JobProgressUI(sc: SparkContext) extends UIComponent { + val listener = new JobProgressListener + val fmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") + + sc.addSparkListener(listener) + + def getHandlers = Seq[(String, Handler)]( + ("/stages/stage", (request: HttpServletRequest) => stagePage(request)), + ("/stages", (request: HttpServletRequest) => indexPage) + ) + + def stagePage(request: HttpServletRequest): Seq[Node] = { + val stageId = request.getParameter("id").toInt + val content = +

    Percentile Metrics

    + + + + + + + + + + {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} + +
    Service TimeRemote Bytes ReadShuffle Bytes Written
    +

    Tasks

    + + + + + + + + + + + + {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} + +
    Task IDService Time (ms)Locality levelWorkerLaunch Time
    ; + WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) + } + + + + def taskRow(info: TaskInfo, metrics: TaskMetrics): Seq[Node] = { + + {info.taskId} + {metrics.executorRunTime} + {info.taskLocality} + {info.hostPort} + {fmt.format(new Date(info.launchTime))} + + } + + def indexPage: Seq[Node] = { + val content = +

    Active Stages

    + + + + + + + + + + + + {listener.activeStages.map(stageRow)} + +
    Stage IDOriginSubmittedDurationTasks: Complete/Total
    +

    Completed Stages

    + + + + + + + + + + + + {listener.completedStages.map(stageRow)} + +
    Stage IDOriginSubmittedDurationTasks: Complete/Total
    ; + + WebUI.headerSparkPage(content, "Spark Stages") + } + + def getElapsedTime(submitted: Option[Long], completed: Long): String = { + submitted match { + case Some(t) => Duration(completed - t, "milliseconds").printHMS + case _ => "Unknown" + } + } + + def stageRow(s: Stage): Seq[Node] = { + val submissionTime = s.submissionTime match { + case Some(t) => fmt.format(new Date(t)) + case None => "Unknown" + } + + {s.id} + {s.origin} + {submissionTime} + {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} + {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} + {listener.stageToTasksFailed.getOrElse(s.id, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => + }} + + + } } -class JobProgressListener extends SparkListener { +private[spark] class JobProgressListener extends SparkListener { + val activeStages = HashSet[Stage]() + val stageToTasksComplete = HashMap[Int, Int]() + val stageToTasksFailed = HashMap[Int, Int]() + val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() + val completedStages = ListBuffer[Stage]() // Todo (pwendell): Evict these over time + override def onJobStart(jobStart: SparkListenerJobStart) { } + + override def onStageCompleted(stageCompleted: StageCompleted) = { + val stage = stageCompleted.stageInfo.stage + activeStages -= stage + stage +=: completedStages + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = + activeStages += stageSubmitted.stage + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val sid = taskEnd.event.task.stageId + taskEnd.event.reason match { + case Success => + stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 + case _ => + stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 + } + val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) + taskList += ((taskEnd.event.taskInfo, taskEnd.event.taskMetrics)) + stageToTaskInfos(sid) = taskList + } + + override def onJobEnd(jobEnd: SparkListenerEvents) { } } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 6c9affc1f7..63b75df36f 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -14,7 +14,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), ("*", (request: HttpServletRequest) => WebUI.headerSparkPage(

    Test

    , "Test page")) ) - val components = Seq(new BlockManagerUI(sc)) + val components = Seq(new BlockManagerUI(sc), new JobProgressUI(sc)) def start() { /** Start an HTTP server to run the Web interface */ diff --git a/core/src/main/scala/spark/ui/WebUI.scala b/core/src/main/scala/spark/ui/WebUI.scala index 16251e727a..e80d48dd4a 100644 --- a/core/src/main/scala/spark/ui/WebUI.scala +++ b/core/src/main/scala/spark/ui/WebUI.scala @@ -58,7 +58,6 @@ object WebUI extends Logging { handler } else { val contextHandler = new ContextHandler(path) - println("Adding handler for path: " + path) contextHandler.setHandler(handler) contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] } @@ -84,7 +83,7 @@ object WebUI extends Logging { /** Page with Spark logo, title, and Spark UI headers */ def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { val newContent = -

    Storage | Jobs

    ; +

    Storage | Jobs

    ; sparkPage(newContent ++ content, title) } From ce81c320acfd265b8de7c30a46c6a3adc4f39a74 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 10:45:52 -0700 Subject: [PATCH 027/419] Adding helper function to make listing tables --- .../main/scala/spark/ui/BlockManagerUI.scala | 155 ++++++++---------- .../main/scala/spark/ui/JobProgressUI.scala | 71 +++----- core/src/main/scala/spark/ui/WebUI.scala | 12 ++ 3 files changed, 97 insertions(+), 141 deletions(-) diff --git a/core/src/main/scala/spark/ui/BlockManagerUI.scala b/core/src/main/scala/spark/ui/BlockManagerUI.scala index f319751590..3be5064837 100644 --- a/core/src/main/scala/spark/ui/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/BlockManagerUI.scala @@ -1,14 +1,15 @@ package spark.ui -import akka.actor.{ActorRef, ActorSystem} import akka.util.Duration import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import spark.{Logging, SparkContext} -import spark.Utils -import WebUI._ +import spark.{RDD, Logging, SparkContext, Utils} +import spark.ui.WebUI._ import xml.Node import spark.storage.StorageUtils +import spark.storage.StorageStatus +import spark.storage.RDDInfo +import spark.storage.BlockManagerMasterActor.BlockStatus /** * Web UI server for the BlockManager inside each SparkContext. @@ -33,6 +34,14 @@ class BlockManagerUI(sc: SparkContext) filterStorageStatusByPrefix(storageStatusList, prefix) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head + val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage") + val workers = filteredStorageStatusList.map((prefix, _)) + val workerTable = listingTable(workerHeaders, workerRow, workers) + + val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk") + val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockTable = listingTable(blockHeaders, blockRow, blocks) + val content =
    @@ -64,67 +73,38 @@ class BlockManagerUI(sc: SparkContext)

    RDD Summary

    -
    - - - - - - - - - - - {filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { - case (k,v) => - - - - - - - } - } - -
    Block NameStorage LevelSize in MemorySize on Disk
    {k} - {v.storageLevel.description} - {Utils.memoryBytesToString(v.memSize)}{Utils.memoryBytesToString(v.diskSize)}
    +
    {blockTable}
    -
    -
    -
    -

    Worker Summary

    -
    - - - - - - - - - - {filteredStorageStatusList.map { - status => - - - - - - } - } - -
    HostMemory UsageDisk Usage
    {status.blockManagerId.host + ":" + status.blockManagerId.port} - {Utils.memoryBytesToString(status.memUsed(prefix))} - ({Utils.memoryBytesToString(status.memRemaining)} Total Available) - {Utils.memoryBytesToString(status.diskUsed(prefix))}
    -
    -
    ; +
    ++ {workerTable}; WebUI.headerSparkPage(content, "RDD Info: " + id) } + def blockRow(blk: (String, BlockStatus)): Seq[Node] = { + val (id, block) = blk + + {id} + + {block.storageLevel.description} + + {Utils.memoryBytesToString(block.memSize)} + {Utils.memoryBytesToString(block.diskSize)} + + } + + def workerRow(worker: (String, StorageStatus)): Seq[Node] = { + val (prefix, status) = worker + + {status.blockManagerId.host + ":" + status.blockManagerId.port} + + {Utils.memoryBytesToString(status.memUsed(prefix))} + ({Utils.memoryBytesToString(status.memRemaining)} Total Available) + + {Utils.memoryBytesToString(status.diskUsed(prefix))} + + } + def indexPage: Seq[Node] = { val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics @@ -132,7 +112,16 @@ class BlockManagerUI(sc: SparkContext) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) + + val rddHeaders = Seq( + "RDD Name", + "Storage Level", + "Cached Partitions", + "Fraction Partitions Cached", + "Size in Memory", + "Size on Disk") val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + val rddTable = listingTable(rddHeaders, rddRow, rdds) val content =
    @@ -144,38 +133,24 @@ class BlockManagerUI(sc: SparkContext)
  • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
  • -
    -
    - - - - - - - - - - - - - {for (rdd <- rdds) yield - - - - - - - - - } - -
    RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
    - - {rdd.name} - - {rdd.storageLevel.description} - {rdd.numCachedPartitions}{rdd.numCachedPartitions / rdd.numPartitions.toDouble}{Utils.memoryBytesToString(rdd.memSize)}{Utils.memoryBytesToString(rdd.diskSize)}
    ; +
    ++ {rddTable}; WebUI.headerSparkPage(content, "Spark Storage ") } + + def rddRow(rdd: RDDInfo): Seq[Node] = { + + + + {rdd.name} + + + {rdd.storageLevel.description} + + {rdd.numCachedPartitions} + {rdd.numCachedPartitions / rdd.numPartitions.toDouble} + {Utils.memoryBytesToString(rdd.memSize)} + {Utils.memoryBytesToString(rdd.diskSize)} + + } } diff --git a/core/src/main/scala/spark/ui/JobProgressUI.scala b/core/src/main/scala/spark/ui/JobProgressUI.scala index 1f12df10b8..b51e62bb1a 100644 --- a/core/src/main/scala/spark/ui/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/JobProgressUI.scala @@ -39,6 +39,11 @@ class JobProgressUI(sc: SparkContext) extends UIComponent { def stagePage(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt + + val taskHeaders = Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") + val tasks = listener.stageToTaskInfos(stageId) + val taskTable = listingTable(taskHeaders, taskRow, tasks) + val content =

    Percentile Metrics

    @@ -53,27 +58,13 @@ class JobProgressUI(sc: SparkContext) extends UIComponent { {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }}
    -

    Tasks

    - - - - - - - - - - - - {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} - -
    Task IDService Time (ms)Locality levelWorkerLaunch Time
    ; +

    Tasks

    ++ {taskTable}; + WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) } - - - def taskRow(info: TaskInfo, metrics: TaskMetrics): Seq[Node] = { + def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { + val (info, metrics) = taskData {info.taskId} {metrics.executorRunTime} @@ -84,37 +75,16 @@ class JobProgressUI(sc: SparkContext) extends UIComponent { } def indexPage: Seq[Node] = { + val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total") + val activeStages = listener.activeStages.toSeq + val completedStages = listener.completedStages.toSeq + + val activeStageTable = listingTable(stageHeaders, stageRow, activeStages) + val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) + val content = -

    Active Stages

    - - - - - - - - - - - - {listener.activeStages.map(stageRow)} - -
    Stage IDOriginSubmittedDurationTasks: Complete/Total
    -

    Completed Stages

    - - - - - - - - - - - - {listener.completedStages.map(stageRow)} - -
    Stage IDOriginSubmittedDurationTasks: Complete/Total
    ; +

    Active Stages

    ++ {activeStageTable} +

    Completed Stages

    ++ {completedStageTable} WebUI.headerSparkPage(content, "Spark Stages") } @@ -139,7 +109,7 @@ class JobProgressUI(sc: SparkContext) extends UIComponent { {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) - case _ => + case _ => }} @@ -178,5 +148,4 @@ private[spark] class JobProgressListener extends SparkListener { } override def onJobEnd(jobEnd: SparkListenerEvents) { } -} - +} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/WebUI.scala b/core/src/main/scala/spark/ui/WebUI.scala index e80d48dd4a..265624a726 100644 --- a/core/src/main/scala/spark/ui/WebUI.scala +++ b/core/src/main/scala/spark/ui/WebUI.scala @@ -14,6 +14,7 @@ abstract class UIComponent { } object WebUI extends Logging { + // CORE WEB UI COMPONENTS type Responder[T] = HttpServletRequest => T implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = @@ -80,6 +81,7 @@ object WebUI extends Logging { connect(port) } + // HELPER FUNCTIONS AND SHORTCUTS /** Page with Spark logo, title, and Spark UI headers */ def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { val newContent = @@ -117,4 +119,14 @@ object WebUI extends Logging { } + + /** Shortcut for making a table derived from a sequence of objects. */ + def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + {headers.map(h => "".format(h))} + + {rows.map(r => makeRow(r))} + +
    %s
    + } } From dcf6a68177912ffa69ee75345c8172fd2db9e0ea Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 12:05:49 -0700 Subject: [PATCH 028/419] Refactoring into different modules --- core/src/main/scala/spark/SparkContext.scala | 2 +- .../main/scala/spark/ui/BlockManagerUI.scala | 156 ------------------ .../main/scala/spark/ui/JobProgressUI.scala | 151 ----------------- core/src/main/scala/spark/ui/SparkUI.scala | 2 + core/src/main/scala/spark/ui/WebUI.scala | 7 +- .../main/scala/spark/ui/jobs/IndexPage.scala | 55 ++++++ .../scala/spark/ui/jobs/JobProgressUI.scala | 82 +++++++++ .../main/scala/spark/ui/jobs/StagePage.scala | 63 +++++++ .../spark/ui/storage/BlockManagerUI.scala | 26 +++ .../scala/spark/ui/storage/IndexPage.scala | 61 +++++++ .../main/scala/spark/ui/storage/RDDPage.scala | 94 +++++++++++ 11 files changed, 390 insertions(+), 309 deletions(-) delete mode 100644 core/src/main/scala/spark/ui/BlockManagerUI.scala delete mode 100644 core/src/main/scala/spark/ui/JobProgressUI.scala create mode 100644 core/src/main/scala/spark/ui/jobs/IndexPage.scala create mode 100644 core/src/main/scala/spark/ui/jobs/JobProgressUI.scala create mode 100644 core/src/main/scala/spark/ui/jobs/StagePage.scala create mode 100644 core/src/main/scala/spark/ui/storage/BlockManagerUI.scala create mode 100644 core/src/main/scala/spark/ui/storage/IndexPage.scala create mode 100644 core/src/main/scala/spark/ui/storage/RDDPage.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 155a5ee721..901cda4174 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -48,7 +48,7 @@ import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import ui.{SparkUI, BlockManagerUI} +import ui.{SparkUI} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark diff --git a/core/src/main/scala/spark/ui/BlockManagerUI.scala b/core/src/main/scala/spark/ui/BlockManagerUI.scala deleted file mode 100644 index 3be5064837..0000000000 --- a/core/src/main/scala/spark/ui/BlockManagerUI.scala +++ /dev/null @@ -1,156 +0,0 @@ -package spark.ui - -import akka.util.Duration -import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler -import spark.{RDD, Logging, SparkContext, Utils} -import spark.ui.WebUI._ -import xml.Node -import spark.storage.StorageUtils -import spark.storage.StorageStatus -import spark.storage.RDDInfo -import spark.storage.BlockManagerMasterActor.BlockStatus - -/** - * Web UI server for the BlockManager inside each SparkContext. - */ -private[spark] -class BlockManagerUI(sc: SparkContext) - extends UIComponent with Logging { - implicit val timeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - - - def getHandlers = Seq[(String, Handler)]( - ("/storage/rdd", (request: HttpServletRequest) => rddPage(request)), - ("/storage", (request: HttpServletRequest) => indexPage) - ) - - def rddPage(request: HttpServletRequest): Seq[Node] = { - val id = request.getParameter("id") - val prefix = "rdd_" + id.toString - val storageStatusList = sc.getExecutorStorageStatus - val filteredStorageStatusList = StorageUtils. - filterStorageStatusByPrefix(storageStatusList, prefix) - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - - val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage") - val workers = filteredStorageStatusList.map((prefix, _)) - val workerTable = listingTable(workerHeaders, workerRow, workers) - - val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk") - val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) - val blockTable = listingTable(blockHeaders, blockRow, blocks) - - val content = -
    -
    -
      -
    • - Storage Level: - {rddInfo.storageLevel.description} -
    • -
    • - Cached Partitions: - {rddInfo.numCachedPartitions} -
    • -
    • - Total Partitions: - {rddInfo.numPartitions} -
    • -
    • - Memory Size: - {Utils.memoryBytesToString(rddInfo.memSize)} -
    • -
    • - Disk Size: - {Utils.memoryBytesToString(rddInfo.diskSize)} -
    • -
    -
    -
    -
    -
    -
    -

    RDD Summary

    -
    {blockTable} -
    -
    -
    ++ {workerTable}; - - WebUI.headerSparkPage(content, "RDD Info: " + id) - } - - def blockRow(blk: (String, BlockStatus)): Seq[Node] = { - val (id, block) = blk - - {id} - - {block.storageLevel.description} - - {Utils.memoryBytesToString(block.memSize)} - {Utils.memoryBytesToString(block.diskSize)} - - } - - def workerRow(worker: (String, StorageStatus)): Seq[Node] = { - val (prefix, status) = worker - - {status.blockManagerId.host + ":" + status.blockManagerId.port} - - {Utils.memoryBytesToString(status.memUsed(prefix))} - ({Utils.memoryBytesToString(status.memRemaining)} Total Available) - - {Utils.memoryBytesToString(status.diskUsed(prefix))} - - } - - def indexPage: Seq[Node] = { - val storageStatusList = sc.getExecutorStorageStatus - // Calculate macro-level statistics - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) - - val rddHeaders = Seq( - "RDD Name", - "Storage Level", - "Cached Partitions", - "Fraction Partitions Cached", - "Size in Memory", - "Size on Disk") - val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val rddTable = listingTable(rddHeaders, rddRow, rdds) - - val content = -
    -
    -
      -
    • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
    • -
    • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
    • -
    -
    -
    ++ {rddTable}; - - WebUI.headerSparkPage(content, "Spark Storage ") - } - - def rddRow(rdd: RDDInfo): Seq[Node] = { - - - - {rdd.name} - - - {rdd.storageLevel.description} - - {rdd.numCachedPartitions} - {rdd.numCachedPartitions / rdd.numPartitions.toDouble} - {Utils.memoryBytesToString(rdd.memSize)} - {Utils.memoryBytesToString(rdd.diskSize)} - - } -} diff --git a/core/src/main/scala/spark/ui/JobProgressUI.scala b/core/src/main/scala/spark/ui/JobProgressUI.scala deleted file mode 100644 index b51e62bb1a..0000000000 --- a/core/src/main/scala/spark/ui/JobProgressUI.scala +++ /dev/null @@ -1,151 +0,0 @@ -package spark.ui - -import spark.{Utils, SparkContext} -import spark.scheduler._ -import spark.scheduler.SparkListenerTaskEnd -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerStageSubmitted -import org.eclipse.jetty.server.Handler -import javax.servlet.http.HttpServletRequest -import xml.Node -import WebUI._ -import collection.mutable._ -import spark.Success -import akka.util.Duration -import java.text.SimpleDateFormat -import java.util.Date -import spark.scheduler.cluster.TaskInfo -import collection.mutable -import org.hsqldb.lib.HashMappedList -import spark.executor.TaskMetrics -import spark.scheduler.SparkListenerTaskEnd -import scala.Some -import spark.scheduler.SparkListenerStageSubmitted -import scala.Seq -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerJobStart - -private[spark] -class JobProgressUI(sc: SparkContext) extends UIComponent { - val listener = new JobProgressListener - val fmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") - - sc.addSparkListener(listener) - - def getHandlers = Seq[(String, Handler)]( - ("/stages/stage", (request: HttpServletRequest) => stagePage(request)), - ("/stages", (request: HttpServletRequest) => indexPage) - ) - - def stagePage(request: HttpServletRequest): Seq[Node] = { - val stageId = request.getParameter("id").toInt - - val taskHeaders = Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") - val tasks = listener.stageToTaskInfos(stageId) - val taskTable = listingTable(taskHeaders, taskRow, tasks) - - val content = -

    Percentile Metrics

    - - - - - - - - - - {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} - -
    Service TimeRemote Bytes ReadShuffle Bytes Written
    -

    Tasks

    ++ {taskTable}; - - WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) - } - - def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { - val (info, metrics) = taskData - - {info.taskId} - {metrics.executorRunTime} - {info.taskLocality} - {info.hostPort} - {fmt.format(new Date(info.launchTime))} - - } - - def indexPage: Seq[Node] = { - val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total") - val activeStages = listener.activeStages.toSeq - val completedStages = listener.completedStages.toSeq - - val activeStageTable = listingTable(stageHeaders, stageRow, activeStages) - val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) - - val content = -

    Active Stages

    ++ {activeStageTable} -

    Completed Stages

    ++ {completedStageTable} - - WebUI.headerSparkPage(content, "Spark Stages") - } - - def getElapsedTime(submitted: Option[Long], completed: Long): String = { - submitted match { - case Some(t) => Duration(completed - t, "milliseconds").printHMS - case _ => "Unknown" - } - } - - def stageRow(s: Stage): Seq[Node] = { - val submissionTime = s.submissionTime match { - case Some(t) => fmt.format(new Date(t)) - case None => "Unknown" - } - - {s.id} - {s.origin} - {submissionTime} - {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} - {listener.stageToTasksFailed.getOrElse(s.id, 0) match { - case f if f > 0 => "(%s failed)".format(f) - case _ => - }} - - - } -} - -private[spark] class JobProgressListener extends SparkListener { - val activeStages = HashSet[Stage]() - val stageToTasksComplete = HashMap[Int, Int]() - val stageToTasksFailed = HashMap[Int, Int]() - val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() - val completedStages = ListBuffer[Stage]() // Todo (pwendell): Evict these over time - - override def onJobStart(jobStart: SparkListenerJobStart) { } - - override def onStageCompleted(stageCompleted: StageCompleted) = { - val stage = stageCompleted.stageInfo.stage - activeStages -= stage - stage +=: completedStages - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = - activeStages += stageSubmitted.stage - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val sid = taskEnd.event.task.stageId - taskEnd.event.reason match { - case Success => - stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - case _ => - stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - } - val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) - taskList += ((taskEnd.event.taskInfo, taskEnd.event.taskMetrics)) - stageToTaskInfos(sid) = taskList - } - - override def onJobEnd(jobEnd: SparkListenerEvents) { } -} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 63b75df36f..a1f6cc60ec 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -1,8 +1,10 @@ package spark.ui +import jobs.JobProgressUI import spark.{Logging, SparkContext, Utils} import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler +import storage.BlockManagerUI import WebUI._ private[spark] class SparkUI(sc: SparkContext) extends Logging { diff --git a/core/src/main/scala/spark/ui/WebUI.scala b/core/src/main/scala/spark/ui/WebUI.scala index 265624a726..6dedd28fc1 100644 --- a/core/src/main/scala/spark/ui/WebUI.scala +++ b/core/src/main/scala/spark/ui/WebUI.scala @@ -13,6 +13,10 @@ abstract class UIComponent { def getHandlers(): Seq[(String, Handler)] } +abstract class View[T] { + def render(request: HttpServletRequest): T +} + object WebUI extends Logging { // CORE WEB UI COMPONENTS type Responder[T] = HttpServletRequest => T @@ -82,6 +86,7 @@ object WebUI extends Logging { } // HELPER FUNCTIONS AND SHORTCUTS + /** Page with Spark logo, title, and Spark UI headers */ def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { val newContent = @@ -123,7 +128,7 @@ object WebUI extends Logging { /** Shortcut for making a table derived from a sequence of objects. */ def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - {headers.map(h => "".format(h))} + {headers.map(h => )} {rows.map(r => makeRow(r))} diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala new file mode 100644 index 0000000000..1740524f49 --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -0,0 +1,55 @@ +package spark.ui.jobs + +import spark.ui.{WebUI, View} +import xml.{NodeSeq, Node} +import spark.ui.WebUI._ +import scala.Some +import akka.util.Duration +import spark.scheduler.Stage +import java.util.Date +import javax.servlet.http.HttpServletRequest + +class IndexPage(parent: JobProgressUI) extends View[Seq[Node]] { + val listener = parent.listener + val dateFmt = parent.dateFmt + + def render(request: HttpServletRequest): Seq[Node] = { + val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total") + val activeStages = listener.activeStages.toSeq + val completedStages = listener.completedStages.toSeq + + val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow, activeStages) + val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) + + val content =

    Active Stages

    ++ activeStageTable ++ +

    Completed Stages

    ++ completedStageTable + + WebUI.headerSparkPage(content, "Spark Stages") + } + + def getElapsedTime(submitted: Option[Long], completed: Long): String = { + submitted match { + case Some(t) => Duration(completed - t, "milliseconds").printHMS + case _ => "Unknown" + } + } + + def stageRow(s: Stage): Seq[Node] = { + val submissionTime = s.submissionTime match { + case Some(t) => dateFmt.format(new Date(t)) + case None => "Unknown" + } + + + + + + + + } +} diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala new file mode 100644 index 0000000000..9ee962bf2e --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -0,0 +1,82 @@ +package spark.ui.jobs + +import spark.{Utils, SparkContext} +import spark.scheduler._ +import spark.scheduler.SparkListenerTaskEnd +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerStageSubmitted +import org.eclipse.jetty.server.Handler +import javax.servlet.http.HttpServletRequest +import xml.Node +import collection.mutable._ +import spark.Success +import akka.util.Duration +import java.text.SimpleDateFormat +import java.util.Date +import spark.scheduler.cluster.TaskInfo +import collection.mutable +import org.hsqldb.lib.HashMappedList +import spark.executor.TaskMetrics +import spark.scheduler.SparkListenerTaskEnd +import scala.Some +import spark.scheduler.SparkListenerStageSubmitted +import scala.Seq +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerJobStart +import spark.ui.{WebUI, UIComponent} +import spark.ui.WebUI._ +import spark.scheduler.SparkListenerTaskEnd +import scala.Some +import spark.scheduler.SparkListenerStageSubmitted +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerJobStart + +private[spark] +class JobProgressUI(sc: SparkContext) extends UIComponent { + val listener = new JobProgressListener + val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") + + sc.addSparkListener(listener) + + val indexPage = new IndexPage(this) + val stagePage = new StagePage(this) + + def getHandlers = Seq[(String, Handler)]( + ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), + ("/stages", (request: HttpServletRequest) => indexPage.render(request)) + ) +} + +private[spark] class JobProgressListener extends SparkListener { + val activeStages = HashSet[Stage]() + val stageToTasksComplete = HashMap[Int, Int]() + val stageToTasksFailed = HashMap[Int, Int]() + val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() + val completedStages = ListBuffer[Stage]() // Todo (pwendell): Evict these over time + + override def onJobStart(jobStart: SparkListenerJobStart) { } + + override def onStageCompleted(stageCompleted: StageCompleted) = { + val stage = stageCompleted.stageInfo.stage + activeStages -= stage + stage +=: completedStages + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = + activeStages += stageSubmitted.stage + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val sid = taskEnd.event.task.stageId + taskEnd.event.reason match { + case Success => + stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 + case _ => + stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 + } + val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) + taskList += ((taskEnd.event.taskInfo, taskEnd.event.taskMetrics)) + stageToTaskInfos(sid) = taskList + } + + override def onJobEnd(jobEnd: SparkListenerEvents) { } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala new file mode 100644 index 0000000000..49f5b1c73c --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -0,0 +1,63 @@ +package spark.ui.jobs + +import javax.servlet.http.HttpServletRequest +import xml.Node +import spark.ui.WebUI._ +import spark.ui.WebUI +import spark.ui.View +import spark.util.Distribution +import spark.scheduler.cluster.TaskInfo +import spark.executor.TaskMetrics +import java.util.Date + +class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { + val listener = parent.listener + val dateFmt = parent.dateFmt + + def render(request: HttpServletRequest): Seq[Node] = { + val stageId = request.getParameter("id").toInt + + val taskHeaders = Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") + val tasks = listener.stageToTaskInfos(stageId) + val taskTable = listingTable(taskHeaders, taskRow, tasks) + + val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} + val serviceQuantiles = "Service Time " ++ Distribution(serviceTimes).get.getQuantiles().map(_.toString) + val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") + + val quantileTable = listingTable(quantileHeaders, Seq(serviceQuantiles), quantileRow) + + val content = +

    Percentile Metrics

    +
    %s
    {h}
    {s.id}{s.origin}{submissionTime}{getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))}{listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} + {listener.stageToTasksFailed.getOrElse(s.id, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => + }} +
    + + + + + + + + + {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} + +
    Service TimeRemote Bytes ReadShuffle Bytes Written
    +

    Tasks

    ++ {taskTable}; + + WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) + } + + def quantileRow(data: Seq[String]) = {data.map(d => d)} + + def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { + val (info, metrics) = taskData + + {info.taskId} + {metrics.executorRunTime} + {info.taskLocality} + {info.hostPort} + {dateFmt.format(new Date(info.launchTime))} + + } + + +} diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala new file mode 100644 index 0000000000..21555384dd --- /dev/null +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -0,0 +1,26 @@ +package spark.ui.storage + +import akka.util.Duration +import javax.servlet.http.HttpServletRequest +import org.eclipse.jetty.server.Handler +import spark.{Logging, SparkContext} +import spark.ui.WebUI._ +import spark.ui.{UIComponent} + +/** + * Web UI server for the BlockManager inside each SparkContext. + */ +private[spark] +class BlockManagerUI(val sc: SparkContext) + extends UIComponent with Logging { + implicit val timeout = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + + val indexPage = new IndexPage(this) + val rddPage = new RDDPage(this) + + def getHandlers = Seq[(String, Handler)]( + ("/storage/rdd", (request: HttpServletRequest) => rddPage.render(request)), + ("/storage", (request: HttpServletRequest) => indexPage.render(request)) + ) +} diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala new file mode 100644 index 0000000000..2f4857d48a --- /dev/null +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -0,0 +1,61 @@ +package spark.ui.storage + +import xml.Node +import spark.storage.{RDDInfo, StorageUtils} +import spark.ui.WebUI._ +import spark.Utils +import spark.ui.{View, WebUI} +import javax.servlet.http.HttpServletRequest + +class IndexPage(parent: BlockManagerUI) extends View[Seq[Node]] { + val sc = parent.sc + + def render(request: HttpServletRequest): Seq[Node] = { + val storageStatusList = sc.getExecutorStorageStatus + // Calculate macro-level statistics + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + + val rddHeaders = Seq( + "RDD Name", + "Storage Level", + "Cached Partitions", + "Fraction Partitions Cached", + "Size in Memory", + "Size on Disk") + val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + val rddTable = listingTable(rddHeaders, rddRow, rdds) + + val content = +
    +
    +
      +
    • Memory: + {Utils.memoryBytesToString(maxMem - remainingMem)} Used + ({Utils.memoryBytesToString(remainingMem)} Available)
    • +
    • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
    • +
    +
    +
    ++ {rddTable}; + + WebUI.headerSparkPage(content, "Spark Storage ") + } + + def rddRow(rdd: RDDInfo): Seq[Node] = { + + + + {rdd.name} + + + {rdd.storageLevel.description} + + {rdd.numCachedPartitions} + {rdd.numCachedPartitions / rdd.numPartitions.toDouble} + {Utils.memoryBytesToString(rdd.memSize)} + {Utils.memoryBytesToString(rdd.diskSize)} + + } +} diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala new file mode 100644 index 0000000000..957c400080 --- /dev/null +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -0,0 +1,94 @@ +package spark.ui.storage + +import javax.servlet.http.HttpServletRequest +import xml.Node +import spark.storage.{StorageStatus, StorageUtils} +import spark.ui.WebUI._ +import spark.Utils +import spark.ui.WebUI +import spark.ui.View +import spark.storage.BlockManagerMasterActor.BlockStatus + + +class RDDPage(parent: BlockManagerUI) extends View[Seq[Node]] { + val sc = parent.sc + + def render(request: HttpServletRequest): Seq[Node] = { + val id = request.getParameter("id") + val prefix = "rdd_" + id.toString + val storageStatusList = sc.getExecutorStorageStatus + val filteredStorageStatusList = StorageUtils. + filterStorageStatusByPrefix(storageStatusList, prefix) + val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head + + val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage") + val workers = filteredStorageStatusList.map((prefix, _)) + val workerTable = listingTable(workerHeaders, workerRow, workers) + + val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk") + val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockTable = listingTable(blockHeaders, blockRow, blocks) + + val content = +
    +
    +
      +
    • + Storage Level: + {rddInfo.storageLevel.description} +
    • +
    • + Cached Partitions: + {rddInfo.numCachedPartitions} +
    • +
    • + Total Partitions: + {rddInfo.numPartitions} +
    • +
    • + Memory Size: + {Utils.memoryBytesToString(rddInfo.memSize)} +
    • +
    • + Disk Size: + {Utils.memoryBytesToString(rddInfo.diskSize)} +
    • +
    +
    +
    +
    +
    +
    +

    RDD Summary

    +
    {blockTable} +
    +
    +
    ++ {workerTable}; + + WebUI.headerSparkPage(content, "RDD Info: " + id) + } + + def blockRow(blk: (String, BlockStatus)): Seq[Node] = { + val (id, block) = blk + + {id} + + {block.storageLevel.description} + + {Utils.memoryBytesToString(block.memSize)} + {Utils.memoryBytesToString(block.diskSize)} + + } + + def workerRow(worker: (String, StorageStatus)): Seq[Node] = { + val (prefix, status) = worker + + {status.blockManagerId.host + ":" + status.blockManagerId.port} + + {Utils.memoryBytesToString(status.memUsed(prefix))} + ({Utils.memoryBytesToString(status.memRemaining)} Total Available) + + {Utils.memoryBytesToString(status.diskUsed(prefix))} + + } +} From 17776323a6ea21170a10169af66e86c4a6cc65fb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 17:46:33 -0700 Subject: [PATCH 029/419] More work on percentile data: --- .../main/scala/spark/ui/jobs/StagePage.scala | 51 +++++++++++-------- 1 file changed, 31 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 49f5b1c73c..64b8c8418c 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -9,6 +9,7 @@ import spark.util.Distribution import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics import java.util.Date +import scala.collection.mutable.ListBuffer class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { val listener = parent.listener @@ -16,37 +17,45 @@ class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt - - val taskHeaders = Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") val tasks = listener.stageToTaskInfos(stageId) + + val hasShuffleRead = tasks.head._2.shuffleReadMetrics.isDefined + val hasShuffleWrite = tasks.head._2.shuffleWriteMetrics.isDefined + + val taskHeaders = + ListBuffer("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") + if (hasShuffleRead) { taskHeaders += "Shuffle Read (bytes)" } + if (hasShuffleWrite) { taskHeaders += "Shuffle Write (bytes)" } + val taskTable = listingTable(taskHeaders, taskRow, tasks) + // TODO(pwendell): Consider factoring this more nicely with the functions in SparkListener val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} - val serviceQuantiles = "Service Time " ++ Distribution(serviceTimes).get.getQuantiles().map(_.toString) - val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") + val serviceQuantiles = "Service Time" +: Distribution(serviceTimes).get.getQuantiles().map(_.toString) - val quantileTable = listingTable(quantileHeaders, Seq(serviceQuantiles), quantileRow) + val shuffleReadSizes = tasks.map{ + case(info, metrics) => metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble} + val shuffleReadQuantiles = "Shuffle Read" +: Distribution(shuffleReadSizes).get.getQuantiles().map(_.toString) + + val shuffleWriteSizes = tasks.map{ + case(info, metrics) => metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble} + val shuffleWriteQuantiles = "Shuffle Write" +: Distribution(shuffleWriteSizes).get.getQuantiles().map(_.toString) + + + val listings: Seq[Seq[String]] = Seq(serviceQuantiles, + if (hasShuffleRead) shuffleReadQuantiles else Nil, + if (hasShuffleWrite) shuffleWriteQuantiles else Nil) + + val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") + val quantileTable = listingTable(quantileHeaders, quantileRow, listings) val content = -

    Percentile Metrics

    - - - - - - - - - - {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} - -
    Service TimeRemote Bytes ReadShuffle Bytes Written
    -

    Tasks

    ++ {taskTable}; +

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) } - def quantileRow(data: Seq[String]) = {data.map(d => d)} + def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { val (info, metrics) = taskData @@ -56,6 +65,8 @@ class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} + {metrics.shuffleReadMetrics.map{m => {m.remoteBytesRead}}.getOrElse("") } + {metrics.shuffleWriteMetrics.map{m => {m.shuffleBytesWritten}}.getOrElse("") } } From 5c872e9ef55a467a8a5e71f0bd12537eac9d447f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 21:06:18 -0700 Subject: [PATCH 030/419] Documentation and some refactoring --- .../spark/deploy/master/MasterWebUI.scala | 14 ++--- .../spark/deploy/worker/WorkerWebUI.scala | 6 +-- .../spark/ui/{WebUI.scala => JettyUI.scala} | 44 +++++++++------- core/src/main/scala/spark/ui/SparkUI.scala | 35 +++++++------ .../main/scala/spark/ui/jobs/IndexPage.scala | 22 +++++--- .../scala/spark/ui/jobs/JobProgressUI.scala | 51 +++++++------------ .../main/scala/spark/ui/jobs/StagePage.scala | 19 ++++--- .../spark/ui/storage/BlockManagerUI.scala | 9 ++-- .../scala/spark/ui/storage/IndexPage.scala | 8 +-- .../main/scala/spark/ui/storage/RDDPage.scala | 12 ++--- 10 files changed, 111 insertions(+), 109 deletions(-) rename core/src/main/scala/spark/ui/{WebUI.scala => JettyUI.scala} (78%) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 328a7cb297..1ec5f77bd7 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -10,11 +10,11 @@ import net.liftweb.json.JsonAST.JValue import org.eclipse.jetty.server.Handler import scala.xml.Node import spark.{Logging, Utils} -import spark.ui.WebUI -import WebUI._ +import spark.ui.JettyUI +import JettyUI._ import spark.deploy._ import spark.deploy.MasterState -import spark.ui.WebUI +import spark.ui.JettyUI /** * Web UI server for the standalone master. @@ -29,11 +29,11 @@ class MasterWebUI(master: ActorRef) extends Logging { def start() { try { - val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, handlers) + val (server, boundPort) = JettyUI.startJettyServer("0.0.0.0", port, handlers) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Master WebUI", e) + logError("Failed to create Master JettyUI", e) System.exit(1) } } @@ -99,7 +99,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {executorTable(app.executors.values.toList)} ; - WebUI.sparkPage(content, "Application Info: " + app.desc.name) + JettyUI.sparkPage(content, "Application Info: " + app.desc.name) } def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -189,7 +189,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {appTable(state.completedApps.sortBy(_.endTime).reverse)} ; - WebUI.sparkPage(content, "Spark Master: " + state.uri) + JettyUI.sparkPage(content, "Spark Master: " + state.uri) } def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index f661d99815..0febb9364c 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -12,8 +12,8 @@ import org.eclipse.jetty.server.Handler import scala.io.Source import spark.{Utils, Logging} import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} -import spark.ui.{WebUI => UtilsWebUI} -import spark.ui.WebUI._ +import spark.ui.{JettyUI => UtilsWebUI} +import spark.ui.JettyUI._ import xml.Node /** @@ -40,7 +40,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Worker WebUI", e) + logError("Failed to create Worker JettyUI", e) System.exit(1) } } diff --git a/core/src/main/scala/spark/ui/WebUI.scala b/core/src/main/scala/spark/ui/JettyUI.scala similarity index 78% rename from core/src/main/scala/spark/ui/WebUI.scala rename to core/src/main/scala/spark/ui/JettyUI.scala index 6dedd28fc1..c3f01073d5 100644 --- a/core/src/main/scala/spark/ui/WebUI.scala +++ b/core/src/main/scala/spark/ui/JettyUI.scala @@ -1,26 +1,26 @@ package spark.ui import annotation.tailrec + import javax.servlet.http.{HttpServletResponse, HttpServletRequest} -import net.liftweb.json._ + +import net.liftweb.json.{JValue, pretty, render} + import org.eclipse.jetty.server.{Server, Request, Handler} import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} + +import scala.util.{Try, Success, Failure} + import spark.Logging -import util.{Try, Success, Failure} + import xml.Node -abstract class UIComponent { - def getHandlers(): Seq[(String, Handler)] -} - -abstract class View[T] { - def render(request: HttpServletRequest): T -} - -object WebUI extends Logging { - // CORE WEB UI COMPONENTS +private[spark] object JettyUI extends Logging { + // Base type for a function that returns something based on an HTTP request. Allows for + // implicit conversion from many types of functions to jetty Handlers. type Responder[T] = HttpServletRequest => T + // Conversions from various types of Responder's to jetty Handlers implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) @@ -30,7 +30,7 @@ object WebUI extends Logging { implicit def textResponderToHandler(responder: Responder[String]): Handler = createHandler(responder, "text/plain") - def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, + private def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, extractFn: T => String = (in: Any) => in.toString): Handler = { new AbstractHandler { def handle(target: String, @@ -46,6 +46,7 @@ object WebUI extends Logging { } } + /** Creates a handler for serving files from a static directory. */ def createStaticHandler(resourceBase: String): ResourceHandler = { val staticHandler = new ResourceHandler Option(getClass.getClassLoader.getResource(resourceBase)) match { @@ -57,6 +58,12 @@ object WebUI extends Logging { staticHandler } + /** + * Attempts to start a Jetty server at the supplied ip:port which uses the supplied handlers. + * + * If the desired port number is contented, continues incrementing ports until a free port is + * found. Returns the chosen port and the jetty Server object. + */ def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { val handlersToRegister = handlers.map { case(path, handler) => if (path == "*") { @@ -84,18 +91,19 @@ object WebUI extends Logging { } connect(port) } +} - // HELPER FUNCTIONS AND SHORTCUTS +/** Utility functions for generating XML pages with spark content. */ +object UIUtils { - /** Page with Spark logo, title, and Spark UI headers */ + /** Returns a page containing the supplied content and the spark web ui headers */ def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { val newContent =

    Storage | Jobs

    ; - sparkPage(newContent ++ content, title) } - /** Page with Spark logo and title */ + /** Returns a page containing the supplied content and the spark css, js, and logo. */ def sparkPage(content: => Seq[Node], title: String): Seq[Node] = { @@ -125,7 +133,7 @@ object WebUI extends Logging { } - /** Shortcut for making a table derived from a sequence of objects. */ + /** Returns an HTML table constructed by generating a row for each object in a sequence. */ def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { {headers.map(h => )} diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index a1f6cc60ec..dd7d33e0fa 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -1,43 +1,46 @@ package spark.ui -import jobs.JobProgressUI -import spark.{Logging, SparkContext, Utils} import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler -import storage.BlockManagerUI -import WebUI._ +import org.eclipse.jetty.server.Handler + +import spark.{Logging, SparkContext, Utils} +import spark.ui.storage.BlockManagerUI +import spark.ui.jobs.JobProgressUI +import spark.ui.UIUtils._ +import spark.ui.JettyUI._ + +/** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { val host = Utils.localHostName() val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt - + var boundPort: Option[Int] = None val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), - ("*", (request: HttpServletRequest) => WebUI.headerSparkPage(

    Test

    , "Test page")) + ("*", (request: HttpServletRequest) => headerSparkPage(

    Test

    , "Test page")) ) - val components = Seq(new BlockManagerUI(sc), new JobProgressUI(sc)) + val storage = new BlockManagerUI(sc) + val jobs = new JobProgressUI(sc) + val allHandlers = handlers ++ storage.getHandlers ++ jobs.getHandlers def start() { /** Start an HTTP server to run the Web interface */ try { - val allHandlers = components.flatMap(_.getHandlers) ++ handlers - val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, allHandlers) - logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort)) + val (server, usedPort) = JettyUI.startJettyServer("0.0.0.0", port, allHandlers) + logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) + boundPort = Some(usedPort) } catch { case e: Exception => - logError("Failed to create Spark WebUI", e) + logError("Failed to create Spark JettyUI", e) System.exit(1) } } - private[spark] def appUIAddress = "http://" + host + ":" + port + private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") } object SparkUI { val DEFAULT_PORT = "33000" val STATIC_RESOURCE_DIR = "spark/webui/static" } - - - diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 1740524f49..811baae811 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -1,15 +1,20 @@ package spark.ui.jobs -import spark.ui.{WebUI, View} -import xml.{NodeSeq, Node} -import spark.ui.WebUI._ -import scala.Some import akka.util.Duration -import spark.scheduler.Stage + import java.util.Date + import javax.servlet.http.HttpServletRequest -class IndexPage(parent: JobProgressUI) extends View[Seq[Node]] { +import scala.Some + +import spark.scheduler.Stage +import spark.ui.UIUtils._ + +import xml.{NodeSeq, Node} + +/** Page showing list of all ongoing and recently finished stages */ +class IndexPage(parent: JobProgressUI) { val listener = parent.listener val dateFmt = parent.dateFmt @@ -24,7 +29,7 @@ class IndexPage(parent: JobProgressUI) extends View[Seq[Node]] { val content =

    Active Stages

    ++ activeStageTable ++

    Completed Stages

    ++ completedStageTable - WebUI.headerSparkPage(content, "Spark Stages") + headerSparkPage(content, "Spark Stages") } def getElapsedTime(submitted: Option[Long], completed: Long): String = { @@ -43,7 +48,8 @@ class IndexPage(parent: JobProgressUI) extends View[Seq[Node]] { - + {data.map(d => )} diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index 21555384dd..d83c826033 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -4,15 +4,12 @@ import akka.util.Duration import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler import spark.{Logging, SparkContext} -import spark.ui.WebUI._ +import spark.ui.JettyUI._ import spark.ui.{UIComponent} -/** - * Web UI server for the BlockManager inside each SparkContext. - */ +/** Web UI showing storage status of all RDD's in the given SparkContext. */ private[spark] -class BlockManagerUI(val sc: SparkContext) - extends UIComponent with Logging { +class BlockManagerUI(val sc: SparkContext) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 2f4857d48a..5ead772bc0 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -2,12 +2,12 @@ package spark.ui.storage import xml.Node import spark.storage.{RDDInfo, StorageUtils} -import spark.ui.WebUI._ import spark.Utils -import spark.ui.{View, WebUI} +import spark.ui.UIUtils._ import javax.servlet.http.HttpServletRequest -class IndexPage(parent: BlockManagerUI) extends View[Seq[Node]] { +/** Page showing list of RDD's currently stored in the cluster */ +class IndexPage(parent: BlockManagerUI) { val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { @@ -40,7 +40,7 @@ class IndexPage(parent: BlockManagerUI) extends View[Seq[Node]] { ++ {rddTable}; - WebUI.headerSparkPage(content, "Spark Storage ") + headerSparkPage(content, "Spark Storage ") } def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 957c400080..7628fde4aa 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -1,16 +1,16 @@ package spark.ui.storage import javax.servlet.http.HttpServletRequest -import xml.Node + import spark.storage.{StorageStatus, StorageUtils} -import spark.ui.WebUI._ +import spark.ui.UIUtils._ import spark.Utils -import spark.ui.WebUI -import spark.ui.View import spark.storage.BlockManagerMasterActor.BlockStatus +import xml.Node -class RDDPage(parent: BlockManagerUI) extends View[Seq[Node]] { +/** Page showing storage details for a given RDD */ +class RDDPage(parent: BlockManagerUI) { val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { @@ -65,7 +65,7 @@ class RDDPage(parent: BlockManagerUI) extends View[Seq[Node]] {
    ++ {workerTable}; - WebUI.headerSparkPage(content, "RDD Info: " + id) + headerSparkPage(content, "RDD Info: " + id) } def blockRow(blk: (String, BlockStatus)): Seq[Node] = { From dd696f3a3d04f5e3b453173c0d4e86690f6bcb1b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 23:20:56 -0700 Subject: [PATCH 031/419] Some renaming and comments --- .../scala/spark/deploy/master/MasterWebUI.scala | 14 +++++++------- .../scala/spark/deploy/worker/WorkerWebUI.scala | 6 +++--- .../spark/ui/{JettyUI.scala => JettyUtils.scala} | 4 +++- core/src/main/scala/spark/ui/SparkUI.scala | 6 +++--- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 2 +- .../scala/spark/ui/storage/BlockManagerUI.scala | 6 ++++-- .../main/scala/spark/ui/storage/IndexPage.scala | 6 ++++-- 7 files changed, 25 insertions(+), 19 deletions(-) rename core/src/main/scala/spark/ui/{JettyUI.scala => JettyUtils.scala} (97%) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 1ec5f77bd7..3238661127 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -10,11 +10,11 @@ import net.liftweb.json.JsonAST.JValue import org.eclipse.jetty.server.Handler import scala.xml.Node import spark.{Logging, Utils} -import spark.ui.JettyUI -import JettyUI._ +import spark.ui.JettyUtils +import JettyUtils._ import spark.deploy._ import spark.deploy.MasterState -import spark.ui.JettyUI +import spark.ui.JettyUtils /** * Web UI server for the standalone master. @@ -29,11 +29,11 @@ class MasterWebUI(master: ActorRef) extends Logging { def start() { try { - val (server, boundPort) = JettyUI.startJettyServer("0.0.0.0", port, handlers) + val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Master JettyUI", e) + logError("Failed to create Master JettyUtils", e) System.exit(1) } } @@ -99,7 +99,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {executorTable(app.executors.values.toList)} ; - JettyUI.sparkPage(content, "Application Info: " + app.desc.name) + JettyUtils.sparkPage(content, "Application Info: " + app.desc.name) } def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -189,7 +189,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {appTable(state.completedApps.sortBy(_.endTime).reverse)} ; - JettyUI.sparkPage(content, "Spark Master: " + state.uri) + JettyUtils.sparkPage(content, "Spark Master: " + state.uri) } def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 0febb9364c..805e7b52db 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -12,8 +12,8 @@ import org.eclipse.jetty.server.Handler import scala.io.Source import spark.{Utils, Logging} import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} -import spark.ui.{JettyUI => UtilsWebUI} -import spark.ui.JettyUI._ +import spark.ui.{JettyUtils => UtilsWebUI} +import spark.ui.JettyUtils._ import xml.Node /** @@ -40,7 +40,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Worker JettyUI", e) + logError("Failed to create Worker JettyUtils", e) System.exit(1) } } diff --git a/core/src/main/scala/spark/ui/JettyUI.scala b/core/src/main/scala/spark/ui/JettyUtils.scala similarity index 97% rename from core/src/main/scala/spark/ui/JettyUI.scala rename to core/src/main/scala/spark/ui/JettyUtils.scala index c3f01073d5..8bb343163a 100644 --- a/core/src/main/scala/spark/ui/JettyUI.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -15,7 +15,9 @@ import spark.Logging import xml.Node -private[spark] object JettyUI extends Logging { + +/** Utilities for launching a web server using Jetty's HTTP Server class */ +private[spark] object JettyUtils extends Logging { // Base type for a function that returns something based on an HTTP request. Allows for // implicit conversion from many types of functions to jetty Handlers. type Responder[T] = HttpServletRequest => T diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index dd7d33e0fa..7428e7a343 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -8,7 +8,7 @@ import spark.{Logging, SparkContext, Utils} import spark.ui.storage.BlockManagerUI import spark.ui.jobs.JobProgressUI import spark.ui.UIUtils._ -import spark.ui.JettyUI._ +import spark.ui.JettyUtils._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { @@ -27,12 +27,12 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { def start() { /** Start an HTTP server to run the Web interface */ try { - val (server, usedPort) = JettyUI.startJettyServer("0.0.0.0", port, allHandlers) + val (server, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) boundPort = Some(usedPort) } catch { case e: Exception => - logError("Failed to create Spark JettyUI", e) + logError("Failed to create Spark JettyUtils", e) System.exit(1) } } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 31aa0f9f0d..97fda84e06 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -9,7 +9,7 @@ import org.eclipse.jetty.server.Handler import scala.Seq import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} -import spark.ui.JettyUI._ +import spark.ui.JettyUtils._ import spark.SparkContext import spark.scheduler._ import spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index d83c826033..4f1928fac9 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -1,11 +1,13 @@ package spark.ui.storage import akka.util.Duration + import javax.servlet.http.HttpServletRequest + import org.eclipse.jetty.server.Handler + import spark.{Logging, SparkContext} -import spark.ui.JettyUI._ -import spark.ui.{UIComponent} +import spark.ui.JettyUtils._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[spark] diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 5ead772bc0..1f9953fb6b 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -1,10 +1,12 @@ package spark.ui.storage -import xml.Node +import javax.servlet.http.HttpServletRequest + import spark.storage.{RDDInfo, StorageUtils} import spark.Utils import spark.ui.UIUtils._ -import javax.servlet.http.HttpServletRequest + +import xml.Node /** Page showing list of RDD's currently stored in the cluster */ class IndexPage(parent: BlockManagerUI) { From 3485e73376fcef524e62547ce3d69bb28a4381ad Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 13:47:27 -0700 Subject: [PATCH 032/419] Style cleanup --- .../scala/spark/deploy/master/Master.scala | 1 + .../spark/deploy/master/MasterWebUI.scala | 266 ------------------ .../deploy/master/ui/ApplicationPage.scala | 100 +++++++ .../spark/deploy/master/ui/IndexPage.scala | 115 ++++++++ .../spark/deploy/master/ui/MasterWebUI.scala | 50 ++++ .../scala/spark/deploy/worker/Worker.scala | 1 + .../spark/deploy/worker/WorkerWebUI.scala | 148 ---------- .../spark/deploy/worker/ui/IndexPage.scala | 97 +++++++ .../spark/deploy/worker/ui/WorkerWebUI.scala | 64 +++++ core/src/main/scala/spark/ui/JettyUtils.scala | 1 - 10 files changed, 428 insertions(+), 415 deletions(-) delete mode 100644 core/src/main/scala/spark/deploy/master/MasterWebUI.scala create mode 100644 core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala create mode 100644 core/src/main/scala/spark/deploy/master/ui/IndexPage.scala create mode 100644 core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala delete mode 100644 core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala create mode 100644 core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala create mode 100644 core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 6a7bbdfcbf..3eca522c7e 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -13,6 +13,7 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import spark.deploy._ import spark.{Logging, SparkException, Utils} import spark.util.AkkaUtils +import ui.MasterWebUI private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala deleted file mode 100644 index 3238661127..0000000000 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ /dev/null @@ -1,266 +0,0 @@ -package spark.deploy.master - -import akka.actor.ActorRef -import akka.dispatch.Await -import akka.pattern.ask -import akka.util.Duration -import akka.util.duration._ -import javax.servlet.http.HttpServletRequest -import net.liftweb.json.JsonAST.JValue -import org.eclipse.jetty.server.Handler -import scala.xml.Node -import spark.{Logging, Utils} -import spark.ui.JettyUtils -import JettyUtils._ -import spark.deploy._ -import spark.deploy.MasterState -import spark.ui.JettyUtils - -/** - * Web UI server for the standalone master. - */ -private[spark] -class MasterWebUI(master: ActorRef) extends Logging { - implicit val timeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - val host = Utils.localHostName() - val port = Option(System.getProperty("master.ui.port")) - .getOrElse(MasterWebUI.DEFAULT_PORT).toInt - - def start() { - try { - val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Master JettyUtils", e) - System.exit(1) - } - } - - val handlers = Array[(String, Handler)]( - ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), - ("/app/json", (request: HttpServletRequest) => appDetailJson(request)), - ("/app", (request: HttpServletRequest) => appDetail(request)), - ("*", (request: HttpServletRequest) => index) - ) - - /** Executor details for a particular application */ - def appDetailJson(request: HttpServletRequest): JValue = { - val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] - val state = Await.result(stateFuture, 3 seconds) - val app = state.activeApps.find(_.id == appId).getOrElse({ - state.completedApps.find(_.id == appId).getOrElse(null) - }) - JsonProtocol.writeApplicationInfo(app) - } - - /** Executor details for a particular application */ - def appDetail(request: HttpServletRequest): Seq[Node] = { - val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] - val state = Await.result(stateFuture, 3 seconds) - val app = state.activeApps.find(_.id == appId).getOrElse({ - state.completedApps.find(_.id == appId).getOrElse(null) - }) - val content = -
    -
    -
    -
      -
    • ID: {app.id}
    • -
    • Description: {app.desc.name}
    • -
    • User: {app.desc.user}
    • -
    • Cores: - { - if (app.desc.maxCores == Integer.MAX_VALUE) { - "Unlimited %s granted".format(app.coresGranted) - } else { - "%s (%s granted, %s left)".format( - app.desc.maxCores, app.coresGranted, app.coresLeft) - } - } -
    • -
    • Memory per Slave: {app.desc.memoryPerSlave}
    • -
    • Submit Date: {app.submitDate}
    • -
    • State: {app.state}
    • -
    • Application Detail UI
    • -
    -
    -
    - -
    - -
    -
    -

    Executor Summary

    -
    - {executorTable(app.executors.values.toList)} -
    -
    ; - JettyUtils.sparkPage(content, "Application Info: " + app.desc.name) - } - - def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { -
    {h}
    {s.id} {s.origin} {submissionTime}{getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))}{getElapsedTime(s.submissionTime, + s.completionTime.getOrElse(System.currentTimeMillis()))} {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 9ee962bf2e..31aa0f9f0d 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -1,45 +1,30 @@ package spark.ui.jobs -import spark.{Utils, SparkContext} -import spark.scheduler._ -import spark.scheduler.SparkListenerTaskEnd -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerStageSubmitted -import org.eclipse.jetty.server.Handler -import javax.servlet.http.HttpServletRequest -import xml.Node -import collection.mutable._ -import spark.Success -import akka.util.Duration import java.text.SimpleDateFormat -import java.util.Date -import spark.scheduler.cluster.TaskInfo -import collection.mutable -import org.hsqldb.lib.HashMappedList -import spark.executor.TaskMetrics -import spark.scheduler.SparkListenerTaskEnd -import scala.Some -import spark.scheduler.SparkListenerStageSubmitted -import scala.Seq -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerJobStart -import spark.ui.{WebUI, UIComponent} -import spark.ui.WebUI._ -import spark.scheduler.SparkListenerTaskEnd -import scala.Some -import spark.scheduler.SparkListenerStageSubmitted -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerJobStart -private[spark] -class JobProgressUI(sc: SparkContext) extends UIComponent { +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.Seq +import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} + +import spark.ui.JettyUI._ +import spark.SparkContext +import spark.scheduler._ +import spark.scheduler.cluster.TaskInfo +import spark.executor.TaskMetrics +import spark.Success + +/** Web UI showing progress status of all jobs in the given SparkContext. */ +private[spark] class JobProgressUI(sc: SparkContext) { val listener = new JobProgressListener val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") sc.addSparkListener(listener) - val indexPage = new IndexPage(this) - val stagePage = new StagePage(this) + private val indexPage = new IndexPage(this) + private val stagePage = new StagePage(this) def getHandlers = Seq[(String, Handler)]( ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 64b8c8418c..74ac811cef 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -1,17 +1,20 @@ package spark.ui.jobs +import java.util.Date + import javax.servlet.http.HttpServletRequest -import xml.Node -import spark.ui.WebUI._ -import spark.ui.WebUI -import spark.ui.View + +import scala.collection.mutable.ListBuffer + +import spark.ui.UIUtils._ import spark.util.Distribution import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics -import java.util.Date -import scala.collection.mutable.ListBuffer -class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { +import xml.Node + +/** Page showing statistics and task list for a given stage */ +class StagePage(parent: JobProgressUI) { val listener = parent.listener val dateFmt = parent.dateFmt @@ -52,7 +55,7 @@ class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { val content =

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; - WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) + headerSparkPage(content, "Stage Details: %s".format(stageId)) } def quantileRow(data: Seq[String]): Seq[Node] =
    {d}
    - - - - - - - - - - - - {executors.map(executorRow)} - -
    ExecutorIDWorkerCoresMemoryStateLogs
    - } - - def executorRow(executor: ExecutorInfo): Seq[Node] = { - - {executor.id} - - {executor.worker.id} - - {executor.cores} - {executor.memory} - {executor.state} - - stdout - stderr - - - } - - /** Index view listing applications and executors */ - def index: Seq[Node] = { - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] - val state = Await.result(stateFuture, 3 seconds) - - val content = -
    -
    -
    -
      -
    • URL:{state.uri}
    • -
    • Workers:{state.workers.size}
    • -
    • Cores: {state.workers.map(_.cores).sum} Total, - {state.workers.map(_.coresUsed).sum} Used
    • -
    • Memory: - {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, - {Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
    • -
    • Applications: - {state.activeApps.size} Running, - {state.completedApps.size} Completed
    • -
    -
    -
    - -
    -
    -

    Workers

    -
    - {workerTable(state.workers.sortBy(_.id))} -
    -
    - -
    - -
    -
    -

    Running Applications

    -
    - {appTable(state.activeApps.sortBy(_.startTime).reverse)} -
    -
    - -
    - -
    -
    -

    Completed Applications

    -
    - {appTable(state.completedApps.sortBy(_.endTime).reverse)} -
    -
    ; - JettyUtils.sparkPage(content, "Spark Master: " + state.uri) - } - - def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { - - - - - - - - - - - - { - workers.map{ worker => - - - - - - - - } - } - -
    IDAddressStateCoresMemory
    - {worker.id} - {worker.host}:{worker.port}{worker.state}{worker.cores} ({worker.coresUsed} Used){Utils.memoryMegabytesToString(worker.memory)} - ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used)
    - } - - def appTable(apps: Seq[spark.deploy.master.ApplicationInfo]) = { - - - - - - - - - - - - - - - { - apps.map{ app => - - - - - - - - - - - } - } - -
    IDDescriptionCoresMemory per NodeSubmit TimeUserStateDuration
    - {app.id} - {app.desc.name} - {app.coresGranted} - {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)}{DeployWebUI.formatDate(app.submitDate)}{app.desc.user}{app.state.toString}{DeployWebUI.formatDuration(app.duration)}
    - } -} - -object MasterWebUI { - val STATIC_RESOURCE_DIR = "spark/webui/static" - val DEFAULT_PORT = "8080" -} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala new file mode 100644 index 0000000000..2a810b71d2 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -0,0 +1,100 @@ +package spark.deploy.master.ui + +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.duration._ + +import javax.servlet.http.HttpServletRequest + +import net.liftweb.json.JsonAST.JValue + +import scala.xml.Node + +import spark.deploy.{RequestMasterState, JsonProtocol, MasterState} +import spark.deploy.master.ExecutorInfo +import spark.ui.UIUtils + +class ApplicationPage(parent: MasterWebUI) { + val master = parent.master + implicit val timeout = parent.timeout + + /** Executor details for a particular application */ + def renderJson(request: HttpServletRequest): JValue = { + val appId = request.getParameter("appId") + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + val app = state.activeApps.find(_.id == appId).getOrElse({ + state.completedApps.find(_.id == appId).getOrElse(null) + }) + JsonProtocol.writeApplicationInfo(app) + } + + /** Executor details for a particular application */ + def render(request: HttpServletRequest): Seq[Node] = { + val appId = request.getParameter("appId") + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + val app = state.activeApps.find(_.id == appId).getOrElse({ + state.completedApps.find(_.id == appId).getOrElse(null) + }) + + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") + val executors = app.executors.values.toSeq + val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) + + val content = +
    +
    +
    +
      +
    • ID: {app.id}
    • +
    • Description: {app.desc.name}
    • +
    • User: {app.desc.user}
    • +
    • Cores: + { + if (app.desc.maxCores == Integer.MAX_VALUE) { + "Unlimited %s granted".format(app.coresGranted) + } else { + "%s (%s granted, %s left)".format( + app.desc.maxCores, app.coresGranted, app.coresLeft) + } + } +
    • +
    • Memory per Slave: {app.desc.memoryPerSlave}
    • +
    • Submit Date: {app.submitDate}
    • +
    • State: {app.state}
    • +
    • Application Detail UI
    • +
    +
    +
    + +
    + +
    +
    +

    Executor Summary

    +
    + {executorTable} +
    +
    ; + UIUtils.sparkPage(content, "Application Info: " + app.desc.name) + } + + def executorRow(executor: ExecutorInfo): Seq[Node] = { + + {executor.id} + + {executor.worker.id} + + {executor.cores} + {executor.memory} + {executor.state} + + stdout + stderr + + + } +} diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala new file mode 100644 index 0000000000..f833c59de8 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -0,0 +1,115 @@ +package spark.deploy.master.ui + +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.duration._ + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import spark.deploy.{RequestMasterState, DeployWebUI, MasterState} +import spark.Utils +import spark.ui.UIUtils +import spark.deploy.master.{ApplicationInfo, WorkerInfo} + +class IndexPage(parent: MasterWebUI) { + val master = parent.master + implicit val timeout = parent.timeout + + /** Index view listing applications and executors */ + def render(request: HttpServletRequest): Seq[Node] = { + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + + val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") + val workers = state.workers.sortBy(_.id) + val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) + + val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User", + "State", "Duration") + val activeApps = state.activeApps.sortBy(_.startTime).reverse + val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) + val completedApps = state.completedApps.sortBy(_.endTime).reverse + val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps) + + val content = +
    +
    +
    +
      +
    • URL:{state.uri}
    • +
    • Workers:{state.workers.size}
    • +
    • Cores: {state.workers.map(_.cores).sum} Total, + {state.workers.map(_.coresUsed).sum} Used
    • +
    • Memory: + {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, + {Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
    • +
    • Applications: + {state.activeApps.size} Running, + {state.completedApps.size} Completed
    • +
    +
    +
    + +
    +
    +

    Workers

    +
    + {workerTable} +
    +
    + +
    + +
    +
    +

    Running Applications

    +
    + {activeAppsTable} +
    +
    + +
    + +
    +
    +

    Completed Applications

    +
    + {completedAppsTable} +
    +
    ; + UIUtils.sparkPage(content, "Spark Master: " + state.uri) + } + + def workerRow(worker: WorkerInfo): Seq[Node] = { + + + {worker.id} + + {worker.host}:{worker.port} + {worker.state} + {worker.cores} ({worker.coresUsed} Used) + {Utils.memoryMegabytesToString(worker.memory)} + ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) + + } + + + def appRow(app: ApplicationInfo): Seq[Node] = { + + + {app.id} + + {app.desc.name} + + {app.coresGranted} + + {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} + {DeployWebUI.formatDate(app.submitDate)} + {app.desc.user} + {app.state.toString} + {DeployWebUI.formatDuration(app.duration)} + + } +} diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala new file mode 100644 index 0000000000..5fd17f10c6 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -0,0 +1,50 @@ +package spark.deploy.master.ui + +import akka.actor.ActorRef +import akka.util.Duration + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import spark.{Logging, Utils} +import spark.ui.JettyUtils +import spark.ui.JettyUtils._ + +/** + * Web UI server for the standalone master. + */ +private[spark] +class MasterWebUI(val master: ActorRef) extends Logging { + implicit val timeout = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val host = Utils.localHostName() + val port = Option(System.getProperty("master.ui.port")) + .getOrElse(MasterWebUI.DEFAULT_PORT).toInt + + val applicationPage = new ApplicationPage(this) + val indexPage = new IndexPage(this) + + def start() { + try { + val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Master JettyUtils", e) + System.exit(1) + } + } + + val handlers = Array[(String, Handler)]( + ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), + ("/app/json", (request: HttpServletRequest) => applicationPage.renderJson(request)), + ("/app", (request: HttpServletRequest) => applicationPage.render(request)), + ("*", (request: HttpServletRequest) => indexPage.render(request)) + ) +} + +object MasterWebUI { + val STATIC_RESOURCE_DIR = "spark/webui/static" + val DEFAULT_PORT = "8080" +} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 3878fe3f7b..690bdfe128 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -14,6 +14,7 @@ import spark.deploy.LaunchExecutor import spark.deploy.RegisterWorkerFailed import spark.deploy.master.Master import java.io.File +import ui.WorkerWebUI private[spark] class Worker( host: String, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala deleted file mode 100644 index 805e7b52db..0000000000 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ /dev/null @@ -1,148 +0,0 @@ -package spark.deploy.worker - -import akka.actor.ActorRef -import akka.dispatch.Await -import akka.pattern.ask -import akka.util.{Duration, Timeout} -import akka.util.duration._ -import java.io.File -import javax.servlet.http.HttpServletRequest -import net.liftweb.json.JsonAST.JValue -import org.eclipse.jetty.server.Handler -import scala.io.Source -import spark.{Utils, Logging} -import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} -import spark.ui.{JettyUtils => UtilsWebUI} -import spark.ui.JettyUtils._ -import xml.Node - -/** - * Web UI server for the standalone worker. - */ -private[spark] -class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { - implicit val timeout = Timeout( - Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) - val host = Utils.localHostName() - val port = Option(System.getProperty("wroker.ui.port")) - .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt - - val handlers = Array[(String, Handler)]( - ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), - ("/log", (request: HttpServletRequest) => log(request)), - ("/json", (request: HttpServletRequest) => indexJson), - ("*", (request: HttpServletRequest) => index) - ) - - def start() { - try { - val (server, boundPort) = UtilsWebUI.startJettyServer("0.0.0.0", port, handlers) - logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Worker JettyUtils", e) - System.exit(1) - } - } - - def indexJson(): JValue = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] - val workerState = Await.result(stateFuture, 3 seconds) - JsonProtocol.writeWorkerState(workerState) - } - - def index(): Seq[Node] = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] - val workerState = Await.result(stateFuture, 3 seconds) - val content = -
    -
    -
    -
      -
    • ID: {workerState.workerId}
    • -
    • - Master URL: {workerState.masterUrl} -
    • -
    • Cores: {workerState.cores} ({workerState.coresUsed} Used)
    • -
    • Memory: {Utils.memoryMegabytesToString(workerState.memory)} - ({Utils.memoryMegabytesToString(workerState.memoryUsed)} Used)
    • -
    -

    Back to Master

    -
    -
    -
    - -
    -
    -

    Running Executors {workerState.executors.size}

    -
    - {executorTable(workerState.executors)} -
    -
    -
    - -
    -
    -

    Finished Executors

    -
    - {executorTable(workerState.finishedExecutors)} -
    -
    ; - - UtilsWebUI.sparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) - } - - def executorTable(executors: Seq[ExecutorRunner]): Seq[Node] = { - - - - - - - - - - - - {executors.map(executorRow)} - -
    ExecutorIDCoresMemoryJob DetailsLogs
    - } - - def executorRow(executor: ExecutorRunner): Seq[Node] = { - - {executor.execId} - {executor.cores} - {Utils.memoryMegabytesToString(executor.memory)} - -
      -
    • ID: {executor.appId}
    • -
    • Name: {executor.appDesc.name}
    • -
    • User: {executor.appDesc.user}
    • -
    - - - stdout - stderr - - - } - - def log(request: HttpServletRequest): String = { - val appId = request.getParameter("appId") - val executorId = request.getParameter("executorId") - val logType = request.getParameter("logType") - val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val source = Source.fromFile(path) - val lines = source.mkString - source.close() - lines - } -} - -object WorkerWebUI { - val STATIC_RESOURCE_DIR = "spark/webui/static" - val DEFAULT_PORT="8081" -} diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala new file mode 100644 index 0000000000..a6aba59e9f --- /dev/null +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -0,0 +1,97 @@ +package spark.deploy.worker.ui + +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.duration._ + +import javax.servlet.http.HttpServletRequest + +import net.liftweb.json.JsonAST.JValue + +import spark.deploy.{RequestWorkerState, JsonProtocol, WorkerState} +import spark.deploy.worker.ExecutorRunner +import spark.Utils +import spark.ui.UIUtils + +import xml.Node + +class IndexPage(parent: WorkerWebUI) { + val worker = parent.worker + val timeout = parent.timeout + + def renderJson(request: HttpServletRequest): JValue = { + val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val workerState = Await.result(stateFuture, 3 seconds) + JsonProtocol.writeWorkerState(workerState) + } + + def render(request: HttpServletRequest): Seq[Node] = { + val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val workerState = Await.result(stateFuture, 3 seconds) + + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") + val runningExecutorTable = + UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) + val finishedExecutorTable = + UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors) + + val content = +
    +
    +
    +
      +
    • ID: {workerState.workerId}
    • +
    • + Master URL: {workerState.masterUrl} +
    • +
    • Cores: {workerState.cores} ({workerState.coresUsed} Used)
    • +
    • Memory: {Utils.memoryMegabytesToString(workerState.memory)} + ({Utils.memoryMegabytesToString(workerState.memoryUsed)} Used)
    • +
    +

    Back to Master

    +
    +
    +
    + +
    +
    +

    Running Executors {workerState.executors.size}

    +
    + {runningExecutorTable} +
    +
    +
    + +
    +
    +

    Finished Executors

    +
    + {finishedExecutorTable} +
    +
    ; + + UIUtils.sparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + } + + def executorRow(executor: ExecutorRunner): Seq[Node] = { + + {executor.execId} + {executor.cores} + {Utils.memoryMegabytesToString(executor.memory)} + +
      +
    • ID: {executor.appId}
    • +
    • Name: {executor.appDesc.name}
    • +
    • User: {executor.appDesc.user}
    • +
    + + + stdout + stderr + + + } + +} diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala new file mode 100644 index 0000000000..abfc847527 --- /dev/null +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -0,0 +1,64 @@ +package spark.deploy.worker.ui + +import akka.actor.ActorRef +import akka.util.{Duration, Timeout} + +import java.io.File + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.io.Source + +import spark.{Utils, Logging} +import spark.ui.JettyUtils +import spark.ui.JettyUtils._ + +/** + * Web UI server for the standalone worker. + */ +private[spark] +class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { + implicit val timeout = Timeout( + Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) + val host = Utils.localHostName() + val port = Option(System.getProperty("wroker.ui.port")) + .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt + + val indexPage = new IndexPage(this) + + val handlers = Array[(String, Handler)]( + ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), + ("/log", (request: HttpServletRequest) => log(request)), + ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), + ("*", (request: HttpServletRequest) => indexPage.render(request)) + ) + + def start() { + try { + val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Worker JettyUtils", e) + System.exit(1) + } + } + + def log(request: HttpServletRequest): String = { + val appId = request.getParameter("appId") + val executorId = request.getParameter("executorId") + val logType = request.getParameter("logType") + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) + val source = Source.fromFile(path) + val lines = source.mkString + source.close() + lines + } +} + +object WorkerWebUI { + val STATIC_RESOURCE_DIR = "spark/webui/static" + val DEFAULT_PORT="8081" +} diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 8bb343163a..93e7129fc0 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -15,7 +15,6 @@ import spark.Logging import xml.Node - /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { // Base type for a function that returns something based on an HTTP request. Allows for From a86bb459e220cfb1375a6f31a94680ab24004b69 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 14:56:57 -0700 Subject: [PATCH 033/419] Showing shuffle status and purging old stages --- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- .../main/scala/spark/ui/jobs/IndexPage.scala | 12 +++++- .../scala/spark/ui/jobs/JobProgressUI.scala | 37 ++++++++++++++++++- .../main/scala/spark/ui/jobs/StagePage.scala | 20 ++++------ 4 files changed, 56 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 7428e7a343..deb6284a66 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -22,7 +22,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) - val allHandlers = handlers ++ storage.getHandlers ++ jobs.getHandlers + val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers def start() { /** Start an HTTP server to run the Web interface */ diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 811baae811..0174e1ddd1 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -19,7 +19,8 @@ class IndexPage(parent: JobProgressUI) { val dateFmt = parent.dateFmt def render(request: HttpServletRequest): Seq[Node] = { - val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total") + val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total", + "Shuffle Activity") val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.toSeq @@ -44,6 +45,14 @@ class IndexPage(parent: JobProgressUI) { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" } + val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) + val shuffleString = (read, write) match { + case (true, true) => "Read/Write" + case (true, false) => "Read" + case (false, true) => "Write" + case _ => "None" + } + {s.id} {s.origin} @@ -56,6 +65,7 @@ class IndexPage(parent: JobProgressUI) { case _ => }} + {shuffleString} } } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 97fda84e06..d260523590 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -33,11 +33,16 @@ private[spark] class JobProgressUI(sc: SparkContext) { } private[spark] class JobProgressListener extends SparkListener { + // TODO(pwendell) Currently does not handle entirely failed stages + + // How many stages to remember + val RETAINED_STAGES = 1000 + val activeStages = HashSet[Stage]() val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() - val completedStages = ListBuffer[Stage]() // Todo (pwendell): Evict these over time + val completedStages = ListBuffer[Stage]() override def onJobStart(jobStart: SparkListenerJobStart) { } @@ -45,6 +50,18 @@ private[spark] class JobProgressListener extends SparkListener { val stage = stageCompleted.stageInfo.stage activeStages -= stage stage +=: completedStages + if (completedStages.size > RETAINED_STAGES) purgeStages() + } + + /** Remove and garbage collect old stages */ + def purgeStages() { + val toRemove = RETAINED_STAGES / 10 + completedStages.takeRight(toRemove).foreach( s => { + stageToTasksComplete.remove(s.id) + stageToTasksFailed.remove(s.id) + stageToTaskInfos.remove(s.id) + }) + completedStages.trimEnd(toRemove) } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = @@ -64,4 +81,22 @@ private[spark] class JobProgressListener extends SparkListener { } override def onJobEnd(jobEnd: SparkListenerEvents) { } + + /** Is this stage's input from a shuffle read. */ + def hasShuffleRead(stageID: Int): Boolean = { + // This is written in a slightly complicated way to avoid having to scan all tasks + for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { + if (s._2 != null) return s._2.shuffleReadMetrics.isDefined + } + return false // No tasks have finished for this stage + } + + /** Is this stage's output to a shuffle write. */ + def hasShuffleWrite(stageID: Int): Boolean = { + // This is written in a slightly complicated way to avoid having to scan all tasks + for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { + if (s._2 != null) return s._2.shuffleWriteMetrics.isDefined + } + return false // No tasks have finished for this stage + } } \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 74ac811cef..3c23a86d12 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -4,8 +4,6 @@ import java.util.Date import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.ListBuffer - import spark.ui.UIUtils._ import spark.util.Distribution import spark.scheduler.cluster.TaskInfo @@ -22,13 +20,13 @@ class StagePage(parent: JobProgressUI) { val stageId = request.getParameter("id").toInt val tasks = listener.stageToTaskInfos(stageId) - val hasShuffleRead = tasks.head._2.shuffleReadMetrics.isDefined - val hasShuffleWrite = tasks.head._2.shuffleWriteMetrics.isDefined + val shuffleRead = listener.hasShuffleRead(stageId) + val shuffleWrite = listener.hasShuffleWrite(stageId) - val taskHeaders = - ListBuffer("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") - if (hasShuffleRead) { taskHeaders += "Shuffle Read (bytes)" } - if (hasShuffleWrite) { taskHeaders += "Shuffle Write (bytes)" } + val taskHeaders: Seq[String] = + Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") ++ + {if (shuffleRead) Seq("Shuffle Read (bytes)") else Nil} ++ + {if (shuffleWrite) Seq("Shuffle Write (bytes)") else Nil} val taskTable = listingTable(taskHeaders, taskRow, tasks) @@ -46,8 +44,8 @@ class StagePage(parent: JobProgressUI) { val listings: Seq[Seq[String]] = Seq(serviceQuantiles, - if (hasShuffleRead) shuffleReadQuantiles else Nil, - if (hasShuffleWrite) shuffleWriteQuantiles else Nil) + if (shuffleRead) shuffleReadQuantiles else Nil, + if (shuffleWrite) shuffleWriteQuantiles else Nil) val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") val quantileTable = listingTable(quantileHeaders, quantileRow, listings) @@ -72,6 +70,4 @@ class StagePage(parent: JobProgressUI) { {metrics.shuffleWriteMetrics.map{m => {m.shuffleBytesWritten}}.getOrElse("") } } - - } From f91e1c48224366c9cf7a32e99d67e966e540b86b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 15:11:42 -0700 Subject: [PATCH 034/419] Linking RDD information when available in stages --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 13 ++++++++++--- core/src/main/scala/spark/ui/storage/RDDPage.scala | 2 +- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 0174e1ddd1..8ebde7a379 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -10,6 +10,7 @@ import scala.Some import spark.scheduler.Stage import spark.ui.UIUtils._ +import spark.storage.StorageLevel import xml.{NodeSeq, Node} @@ -20,7 +21,7 @@ class IndexPage(parent: JobProgressUI) { def render(request: HttpServletRequest): Seq[Node] = { val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total", - "Shuffle Activity") + "Shuffle Activity", "RDDs") val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.toSeq @@ -46,7 +47,7 @@ class IndexPage(parent: JobProgressUI) { case None => "Unknown" } val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) - val shuffleString = (read, write) match { + val shuffleInfo = (read, write) match { case (true, true) => "Read/Write" case (true, false) => "Read" case (false, true) => "Write" @@ -65,7 +66,13 @@ class IndexPage(parent: JobProgressUI) { case _ => }} - {shuffleString} + {shuffleInfo} + {if (s.rdd.getStorageLevel != StorageLevel.NONE) { + + {Option(s.rdd.name).getOrElse(s.rdd.id)} + + }} + } } diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 7628fde4aa..466f4643bc 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -65,7 +65,7 @@ class RDDPage(parent: BlockManagerUI) {
    ++ {workerTable}; - headerSparkPage(content, "RDD Info: " + id) + headerSparkPage(content, "RDD Info: " + rddInfo.name) } def blockRow(blk: (String, BlockStatus)): Seq[Node] = { From 9a24d1a2d0deb5cbd96b806a1ef3afb9353872dd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 15:17:50 -0700 Subject: [PATCH 035/419] Using scala in XML imports --- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/JettyUtils.scala | 3 +-- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 3 +-- core/src/main/scala/spark/ui/jobs/StagePage.scala | 4 ++-- core/src/main/scala/spark/ui/storage/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/storage/RDDPage.scala | 4 ++-- 6 files changed, 10 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index a6aba59e9f..9e7b2bfd92 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -8,13 +8,13 @@ import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue +import scala.xml.Node + import spark.deploy.{RequestWorkerState, JsonProtocol, WorkerState} import spark.deploy.worker.ExecutorRunner import spark.Utils import spark.ui.UIUtils -import xml.Node - class IndexPage(parent: WorkerWebUI) { val worker = parent.worker val timeout = parent.timeout diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 93e7129fc0..726a95594b 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -10,11 +10,10 @@ import org.eclipse.jetty.server.{Server, Request, Handler} import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} import scala.util.{Try, Success, Failure} +import scala.xml.Node import spark.Logging -import xml.Node - /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { // Base type for a function that returns something based on an HTTP request. Allows for diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 8ebde7a379..87f3abc6b0 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -7,13 +7,12 @@ import java.util.Date import javax.servlet.http.HttpServletRequest import scala.Some +import scala.xml.{NodeSeq, Node} import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.storage.StorageLevel -import xml.{NodeSeq, Node} - /** Page showing list of all ongoing and recently finished stages */ class IndexPage(parent: JobProgressUI) { val listener = parent.listener diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 3c23a86d12..d76c07e458 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -4,13 +4,13 @@ import java.util.Date import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import spark.ui.UIUtils._ import spark.util.Distribution import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics -import xml.Node - /** Page showing statistics and task list for a given stage */ class StagePage(parent: JobProgressUI) { val listener = parent.listener diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 1f9953fb6b..52b9087099 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -2,12 +2,12 @@ package spark.ui.storage import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import spark.storage.{RDDInfo, StorageUtils} import spark.Utils import spark.ui.UIUtils._ -import xml.Node - /** Page showing list of RDD's currently stored in the cluster */ class IndexPage(parent: BlockManagerUI) { val sc = parent.sc diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 466f4643bc..1031b2e17f 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -2,13 +2,13 @@ package spark.ui.storage import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import spark.storage.{StorageStatus, StorageUtils} import spark.ui.UIUtils._ import spark.Utils import spark.storage.BlockManagerMasterActor.BlockStatus -import xml.Node - /** Page showing storage details for a given RDD */ class RDDPage(parent: BlockManagerUI) { val sc = parent.sc From be6107ce446a25f6fccaf34085beb23e81828495 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 21:30:56 -0700 Subject: [PATCH 036/419] Some tweaking with shared page header --- core/src/main/scala/spark/ui/JettyUtils.scala | 15 ++++++++++++--- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- .../main/scala/spark/ui/storage/IndexPage.scala | 2 +- .../src/main/scala/spark/ui/storage/RDDPage.scala | 2 +- 7 files changed, 18 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 726a95594b..225daf5778 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -12,7 +12,7 @@ import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHa import scala.util.{Try, Success, Failure} import scala.xml.Node -import spark.Logging +import spark.{Utils, SparkContext, Logging} /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { @@ -97,9 +97,18 @@ private[spark] object JettyUtils extends Logging { object UIUtils { /** Returns a page containing the supplied content and the spark web ui headers */ - def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { + def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String): Seq[Node] = { val newContent = -

    Storage | Jobs

    ; +
    +
    +
      +
    • Master: {sc.master}
    • +
    • Application: {sc.appName}
    • +
    +

    Storage | Jobs

    +
    +
    +
    ; sparkPage(newContent ++ content, title) } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index deb6284a66..cfa805fcf8 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -18,7 +18,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), - ("*", (request: HttpServletRequest) => headerSparkPage(

    Test

    , "Test page")) + ("*", (request: HttpServletRequest) => headerSparkPage(

    Test

    , sc, "Test page")) ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 87f3abc6b0..5a4658499a 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -30,7 +30,7 @@ class IndexPage(parent: JobProgressUI) { val content =

    Active Stages

    ++ activeStageTable ++

    Completed Stages

    ++ completedStageTable - headerSparkPage(content, "Spark Stages") + headerSparkPage(content, parent.sc, "Spark Stages") } def getElapsedTime(submitted: Option[Long], completed: Long): String = { diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index d260523590..47a6bcc2c3 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -17,7 +17,7 @@ import spark.executor.TaskMetrics import spark.Success /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[spark] class JobProgressUI(sc: SparkContext) { +private[spark] class JobProgressUI(val sc: SparkContext) { val listener = new JobProgressListener val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index d76c07e458..42eb1f9eef 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -53,7 +53,7 @@ class StagePage(parent: JobProgressUI) { val content =

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; - headerSparkPage(content, "Stage Details: %s".format(stageId)) + headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId)) } def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 52b9087099..cd7f6bd9f2 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -42,7 +42,7 @@ class IndexPage(parent: BlockManagerUI) { ++ {rddTable}; - headerSparkPage(content, "Spark Storage ") + headerSparkPage(content, parent.sc, "Spark Storage ") } def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 1031b2e17f..059affedbd 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -65,7 +65,7 @@ class RDDPage(parent: BlockManagerUI) {
    ++ {workerTable}; - headerSparkPage(content, "RDD Info: " + rddInfo.name) + headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name) } def blockRow(blk: (String, BlockStatus)): Seq[Node] = { From 3b7ebdeeb86868ff1ebd269c2ed832012f22e0d1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 22 Jun 2013 09:00:42 -0700 Subject: [PATCH 037/419] Handling entirely failed stages --- .../main/scala/spark/ui/jobs/IndexPage.scala | 18 +++++--- .../scala/spark/ui/jobs/JobProgressUI.scala | 43 ++++++++++++------- .../main/scala/spark/ui/storage/RDDPage.scala | 19 +++++--- 3 files changed, 50 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 5a4658499a..2e2dcdfbc6 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -20,15 +20,18 @@ class IndexPage(parent: JobProgressUI) { def render(request: HttpServletRequest): Seq[Node] = { val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total", - "Shuffle Activity", "RDDs") + "Shuffle Activity", "Stored RDD") val activeStages = listener.activeStages.toSeq - val completedStages = listener.completedStages.toSeq + val completedStages = listener.completedStages.reverse.toSeq + val failedStages = listener.failedStages.reverse.toSeq - val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow, activeStages) - val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) + val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow(), activeStages) + val completedStageTable = listingTable(stageHeaders, stageRow(), completedStages) + val failedStageTable: NodeSeq = listingTable(stageHeaders, stageRow(false), failedStages) val content =

    Active Stages

    ++ activeStageTable ++ -

    Completed Stages

    ++ completedStageTable +

    Completed Stages

    ++ completedStageTable ++ +

    Failed Stages

    ++ failedStageTable headerSparkPage(content, parent.sc, "Spark Stages") } @@ -40,7 +43,7 @@ class IndexPage(parent: JobProgressUI) { } } - def stageRow(s: Stage): Seq[Node] = { + def stageRow(showLink: Boolean = true)(s: Stage): Seq[Node] = { val submissionTime = s.submissionTime match { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" @@ -54,7 +57,8 @@ class IndexPage(parent: JobProgressUI) { } - {s.id} + {if (showLink) {{s.id}} + else {{s.id}}} {s.origin} {submissionTime} {getElapsedTime(s.submissionTime, diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 47a6bcc2c3..dea4f0e3b0 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -33,35 +33,37 @@ private[spark] class JobProgressUI(val sc: SparkContext) { } private[spark] class JobProgressListener extends SparkListener { - // TODO(pwendell) Currently does not handle entirely failed stages - // How many stages to remember val RETAINED_STAGES = 1000 val activeStages = HashSet[Stage]() + val completedStages = ListBuffer[Stage]() + val failedStages = ListBuffer[Stage]() + val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() - val completedStages = ListBuffer[Stage]() - override def onJobStart(jobStart: SparkListenerJobStart) { } + override def onJobStart(jobStart: SparkListenerJobStart) {} override def onStageCompleted(stageCompleted: StageCompleted) = { val stage = stageCompleted.stageInfo.stage activeStages -= stage - stage +=: completedStages - if (completedStages.size > RETAINED_STAGES) purgeStages() + completedStages += stage + trimIfNecessary(completedStages) } - /** Remove and garbage collect old stages */ - def purgeStages() { - val toRemove = RETAINED_STAGES / 10 - completedStages.takeRight(toRemove).foreach( s => { - stageToTasksComplete.remove(s.id) - stageToTasksFailed.remove(s.id) - stageToTaskInfos.remove(s.id) - }) - completedStages.trimEnd(toRemove) + /** If stages is too large, remove and garbage collect old stages */ + def trimIfNecessary(stages: ListBuffer[Stage]) { + if (stages.size > RETAINED_STAGES) { + val toRemove = RETAINED_STAGES / 10 + stages.takeRight(toRemove).foreach( s => { + stageToTasksComplete.remove(s.id) + stageToTasksFailed.remove(s.id) + stageToTaskInfos.remove(s.id) + }) + stages.trimEnd(toRemove) + } } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = @@ -80,7 +82,16 @@ private[spark] class JobProgressListener extends SparkListener { stageToTaskInfos(sid) = taskList } - override def onJobEnd(jobEnd: SparkListenerEvents) { } + override def onJobEnd(jobEnd: SparkListenerEvents) { + jobEnd match { + case failed: SparkListenerJobFailed => + val stage = failed.failedStage + activeStages -= stage + failedStages += stage + trimIfNecessary(failedStages) + case _ => + } + } /** Is this stage's input from a shuffle read. */ def hasShuffleRead(stageID: Int): Boolean = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 059affedbd..73f151959e 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -56,14 +56,19 @@ class RDDPage(parent: BlockManagerUI) { -
    -
    -
    -

    RDD Summary

    -
    {blockTable} -
    +
    +
    +
    + {workerTable}
    -
    ++ {workerTable}; +
    +
    +
    +
    +

    RDD Summary

    + {blockTable} +
    +
    ; headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name) } From 7e9f1ed0decbbb261432dcbcd2179b829dbcbc82 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 22 Jun 2013 10:17:29 -0700 Subject: [PATCH 038/419] Some cleanup of styling --- core/src/main/scala/spark/ui/JettyUtils.scala | 45 +++++++++++++------ core/src/main/scala/spark/ui/SparkUI.scala | 4 +- 2 files changed, 35 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 225daf5778..96d773979f 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -41,11 +41,25 @@ private[spark] object JettyUtils extends Logging { response.setStatus(HttpServletResponse.SC_OK) baseRequest.setHandled(true) val result = responder(request) + response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.getWriter().println(extractFn(result)) } } } + def createRedirectHandler(newPath: String): Handler = { + new AbstractHandler { + def handle(target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse) { + response.setStatus(302) + response.setHeader("Location", baseRequest.getRootURL + newPath) + baseRequest.setHandled(true) + } + } + } + /** Creates a handler for serving files from a static directory. */ def createStaticHandler(resourceBase: String): ResourceHandler = { val staticHandler = new ResourceHandler @@ -66,13 +80,9 @@ private[spark] object JettyUtils extends Logging { */ def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { val handlersToRegister = handlers.map { case(path, handler) => - if (path == "*") { - handler - } else { - val contextHandler = new ContextHandler(path) - contextHandler.setHandler(handler) - contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] - } + val contextHandler = new ContextHandler(path) + contextHandler.setHandler(handler) + contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] } val handlerList = new HandlerList @@ -99,13 +109,21 @@ object UIUtils { /** Returns a page containing the supplied content and the spark web ui headers */ def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String): Seq[Node] = { val newContent = -
    -
    +
    +
    +
    + +
    +
    +
    • Master: {sc.master}
    • Application: {sc.appName}
    • +
    • Executors: {sc.getExecutorStorageStatus.size}
    -

    Storage | Jobs


    ; @@ -128,10 +146,11 @@ object UIUtils {
    -
    +
    -

    +

    +
    +

    {title}

    diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index cfa805fcf8..66dcf07384 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -12,13 +12,15 @@ import spark.ui.JettyUtils._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { + // TODO(pwendell): It would be nice to add a view that prints out environment information + val host = Utils.localHostName() val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), - ("*", (request: HttpServletRequest) => headerSparkPage(

    Test

    , sc, "Test page")) + ("/", createRedirectHandler("/stages")) ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) From 8955787a596216a35ad4ec52b57331aa40444bef Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Mon, 24 Jun 2013 09:15:17 +0100 Subject: [PATCH 039/419] Twitter API v1 is retired - username/password auth no longer possible --- .../scala/spark/streaming/StreamingContext.scala | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index f97e47ada0..05be6bd58a 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import twitter4j.Status -import twitter4j.auth.{Authorization, BasicAuthorization} +import twitter4j.auth.Authorization /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -361,20 +361,6 @@ class StreamingContext private ( fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString) } - /** - * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - username: String, - password: String, - filters: Seq[String] = Nil, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[Status] = twitterStream(new BasicAuthorization(username, password), filters, storageLevel) - /** * Create a input stream that returns tweets received from Twitter. * @param twitterAuth Twitter4J authentication From f6e64b5cd6e6ac5ae3bee05af2832e1f71992310 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 22 Jun 2013 10:40:41 -0700 Subject: [PATCH 040/419] Updating based on changes to JobLogger (and one small change to JobLogger) --- .../scala/spark/scheduler/DAGScheduler.scala | 6 +++--- .../scala/spark/scheduler/JobLogger.scala | 2 +- .../scala/spark/scheduler/JobResult.scala | 2 +- .../scala/spark/scheduler/JobWaiter.scala | 2 +- .../scala/spark/ui/jobs/JobProgressUI.scala | 21 +++++++++++-------- 5 files changed, 18 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bdd8792ce9..82d419453b 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -256,7 +256,7 @@ class DAGScheduler( eventQueue.put(toSubmit) waiter.awaitResult() match { case JobSucceeded => {} - case JobFailed(exception: Exception) => + case JobFailed(exception: Exception, _) => logInfo("Failed to run " + callSite) throw exception } @@ -324,7 +324,7 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error, None)))) } return true } @@ -671,7 +671,7 @@ class DAGScheduler( val job = resultStageToJob(resultStage) val error = new SparkException("Job failed: " + reason) job.listener.jobFailed(error) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage))))) activeJobs -= job resultStageToJob -= resultStage } diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 178bfaba3d..6a9d52f356 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -275,7 +275,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { var info = "JOB_ID=" + job.runId reason match { case JobSucceeded => info += " STATUS=SUCCESS" - case JobFailed(exception) => + case JobFailed(exception, _) => info += " STATUS=FAILED REASON=" exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala index 654131ee84..a0fdf391e6 100644 --- a/core/src/main/scala/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/spark/scheduler/JobResult.scala @@ -6,4 +6,4 @@ package spark.scheduler private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult -private[spark] case class JobFailed(exception: Exception) extends JobResult +private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage]) extends JobResult diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala index 3cc6a86345..6ff2e29434 100644 --- a/core/src/main/scala/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala @@ -35,7 +35,7 @@ private[spark] class JobWaiter[T](totalTasks: Int, resultHandler: (Int, T) => Un throw new UnsupportedOperationException("jobFailed() called on a finished JobWaiter") } jobFinished = true - jobResult = JobFailed(exception) + jobResult = JobFailed(exception, None) this.notifyAll() } } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index dea4f0e3b0..70f8e431f2 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -70,25 +70,28 @@ private[spark] class JobProgressListener extends SparkListener { activeStages += stageSubmitted.stage override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val sid = taskEnd.event.task.stageId - taskEnd.event.reason match { + val sid = taskEnd.task.stageId + taskEnd.reason match { case Success => stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 case _ => stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 } val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) - taskList += ((taskEnd.event.taskInfo, taskEnd.event.taskMetrics)) + taskList += ((taskEnd.taskInfo, taskEnd.taskMetrics)) stageToTaskInfos(sid) = taskList } - override def onJobEnd(jobEnd: SparkListenerEvents) { + override def onJobEnd(jobEnd: SparkListenerJobEnd) { jobEnd match { - case failed: SparkListenerJobFailed => - val stage = failed.failedStage - activeStages -= stage - failedStages += stage - trimIfNecessary(failedStages) + case end: SparkListenerJobEnd => + end.jobResult match { + case JobFailed(ex, Some(stage)) => + activeStages -= stage + failedStages += stage + trimIfNecessary(failedStages) + case _ => + } case _ => } } From 93e8ed85aa1fbcb6428934b30d01f2b4090538b9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 13:11:18 -0700 Subject: [PATCH 041/419] Work around for initalization issue --- core/src/main/scala/spark/SparkContext.scala | 5 +++-- core/src/main/scala/spark/ui/SparkUI.scala | 10 +++++++++- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 9 +++++++-- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- 5 files changed, 21 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 901cda4174..1f420d73f7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -102,6 +102,9 @@ class SparkContext( private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]] private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) + // Initalize the Spark UI + private[spark] val ui = new SparkUI(this) + ui.bind() // Add each JAR given through the constructor if (jars != null) { @@ -211,8 +214,6 @@ class SparkContext( @volatile private var dagScheduler = new DAGScheduler(taskScheduler) dagScheduler.start() - // Start the Spark UI - private[spark] val ui = new SparkUI(this) ui.start() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 66dcf07384..2d5a328015 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -26,7 +26,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val jobs = new JobProgressUI(sc) val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers - def start() { + def bind() { /** Start an HTTP server to run the Web interface */ try { val (server, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) @@ -38,6 +38,14 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { System.exit(1) } } + /** Initialize all components of the server */ + def start() { + // NOTE: This is decoupled from bind() because of the following dependency cycle: + // DAGScheduler() requires that the port of this server is known + // This server must register all handlers, including JobProgressUI, before binding + // JobProgressUI registers a listener with SparkContext, which requires sc to initialize + jobs.start() + } private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 2e2dcdfbc6..134c93091d 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -15,7 +15,7 @@ import spark.storage.StorageLevel /** Page showing list of all ongoing and recently finished stages */ class IndexPage(parent: JobProgressUI) { - val listener = parent.listener + def listener = parent.listener val dateFmt = parent.dateFmt def render(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 70f8e431f2..99f9f2d9f6 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -18,10 +18,15 @@ import spark.Success /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { - val listener = new JobProgressListener + private var _listener: Option[JobProgressListener] = None + def listener = _listener.get + val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") - sc.addSparkListener(listener) + def start() { + _listener = Some(new JobProgressListener) + sc.addSparkListener(listener) + } private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 42eb1f9eef..8a488498d9 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -13,7 +13,7 @@ import spark.executor.TaskMetrics /** Page showing statistics and task list for a given stage */ class StagePage(parent: JobProgressUI) { - val listener = parent.listener + def listener = parent.listener val dateFmt = parent.dateFmt def render(request: HttpServletRequest): Seq[Node] = { From b5e6e8bcc8ca05cd97f35a502a89c686aa4a5a12 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 14:13:24 -0700 Subject: [PATCH 042/419] Cleaning up some code for Job Progress --- .../main/scala/spark/ui/jobs/IndexPage.scala | 4 +- .../scala/spark/ui/jobs/JobProgressUI.scala | 9 +++-- .../main/scala/spark/ui/jobs/StagePage.scala | 37 ++++++++++++------- 3 files changed, 31 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 134c93091d..c7ee9dc1c1 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -38,7 +38,7 @@ class IndexPage(parent: JobProgressUI) { def getElapsedTime(submitted: Option[Long], completed: Long): String = { submitted match { - case Some(t) => Duration(completed - t, "milliseconds").printHMS + case Some(t) => parent.formatDuration(completed - t) case _ => "Unknown" } } @@ -53,7 +53,7 @@ class IndexPage(parent: JobProgressUI) { case (true, true) => "Read/Write" case (true, false) => "Read" case (false, true) => "Write" - case _ => "None" + case _ => "" } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 99f9f2d9f6..027eadde3a 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -1,5 +1,7 @@ package spark.ui.jobs +import akka.util.Duration + import java.text.SimpleDateFormat import javax.servlet.http.HttpServletRequest @@ -20,16 +22,17 @@ import spark.Success private[spark] class JobProgressUI(val sc: SparkContext) { private var _listener: Option[JobProgressListener] = None def listener = _listener.get - val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") + private val indexPage = new IndexPage(this) + private val stagePage = new StagePage(this) + def start() { _listener = Some(new JobProgressListener) sc.addSparkListener(listener) } - private val indexPage = new IndexPage(this) - private val stagePage = new StagePage(this) + def formatDuration(ms: Long) = Duration(ms, "milliseconds").printHMS def getHandlers = Seq[(String, Handler)]( ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 8a488498d9..65dbd389b1 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -8,6 +8,7 @@ import scala.xml.Node import spark.ui.UIUtils._ import spark.util.Distribution +import spark.Utils import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics @@ -24,24 +25,30 @@ class StagePage(parent: JobProgressUI) { val shuffleWrite = listener.hasShuffleWrite(stageId) val taskHeaders: Seq[String] = - Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") ++ - {if (shuffleRead) Seq("Shuffle Read (bytes)") else Nil} ++ - {if (shuffleWrite) Seq("Shuffle Write (bytes)") else Nil} + Seq("Task ID", "Service Time", "Locality Level", "Worker", "Launch Time") ++ + {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ + {if (shuffleWrite) Seq("Shuffle Write") else Nil} val taskTable = listingTable(taskHeaders, taskRow, tasks) - // TODO(pwendell): Consider factoring this more nicely with the functions in SparkListener val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} - val serviceQuantiles = "Service Time" +: Distribution(serviceTimes).get.getQuantiles().map(_.toString) + val serviceQuantiles = "Service Time" +: Distribution(serviceTimes).get.getQuantiles().map( + ms => parent.formatDuration(ms.toLong)) - val shuffleReadSizes = tasks.map{ - case(info, metrics) => metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble} - val shuffleReadQuantiles = "Shuffle Read" +: Distribution(shuffleReadSizes).get.getQuantiles().map(_.toString) + def getQuantileCols(data: Seq[Double]) = + Distribution(data).get.getQuantiles().map(d => Utils.memoryBytesToString(d.toLong)) - val shuffleWriteSizes = tasks.map{ - case(info, metrics) => metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble} - val shuffleWriteQuantiles = "Shuffle Write" +: Distribution(shuffleWriteSizes).get.getQuantiles().map(_.toString) + val shuffleReadSizes = tasks.map { + case(info, metrics) => + metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + } + val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) + val shuffleWriteSizes = tasks.map { + case(info, metrics) => + metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + } + val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) val listings: Seq[Seq[String]] = Seq(serviceQuantiles, if (shuffleRead) shuffleReadQuantiles else Nil, @@ -62,12 +69,14 @@ class StagePage(parent: JobProgressUI) { val (info, metrics) = taskData {info.taskId} - {metrics.executorRunTime} + {parent.formatDuration(metrics.executorRunTime)} {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} - {metrics.shuffleReadMetrics.map{m => {m.remoteBytesRead}}.getOrElse("") } - {metrics.shuffleWriteMetrics.map{m => {m.shuffleBytesWritten}}.getOrElse("") } + {metrics.shuffleReadMetrics.map{m => + {Utils.memoryBytesToString(m.remoteBytesRead)}}.getOrElse("") } + {metrics.shuffleWriteMetrics.map{m => + {Utils.memoryBytesToString(m.shuffleBytesWritten)}}.getOrElse("") } } } From a4248138b4c4b374e61f474886463c58f5bb3915 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 14:22:28 -0700 Subject: [PATCH 043/419] Minor style cleanup --- core/src/main/scala/spark/ui/JettyUtils.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 96d773979f..b5270e6062 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -47,6 +47,7 @@ private[spark] object JettyUtils extends Logging { } } + /** Creates a handler that always redirects the user to a given path */ def createRedirectHandler(newPath: String): Handler = { new AbstractHandler { def handle(target: String, @@ -151,7 +152,7 @@ object UIUtils {

    - {title} + {title}

    From 4cda8f865a003ab354890c4915ea1b5a7674f5b0 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 15:14:48 -0700 Subject: [PATCH 044/419] Add simple usage to start-slave script --- bin/start-slave.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 26b5b9d462..1082c09eb1 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -1,4 +1,7 @@ #!/usr/bin/env bash +# +# Usage: start-slave.sh +# where is like "spark://localhost:7077" bin=`dirname "$0"` bin=`cd "$bin"; pwd` From 0bcaf036050c3d2b4389339927239e0e35bf02ff Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 15:39:52 -0700 Subject: [PATCH 045/419] Split out source distro CLASSPATH logic to a separate script --- run | 123 +++++++------------------------------------ set-dev-classpath.sh | 112 +++++++++++++++++++++++++++++++++++++++ 2 files changed, 131 insertions(+), 104 deletions(-) create mode 100644 set-dev-classpath.sh diff --git a/run b/run index c0065c53f1..30a2885a4d 100755 --- a/run +++ b/run @@ -1,7 +1,5 @@ #!/bin/bash -SCALA_VERSION=2.9.3 - # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" @@ -46,36 +44,6 @@ case "$1" in ;; esac -if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then - if [ "$SCALA_HOME" ]; then - RUNNER="${SCALA_HOME}/bin/scala" - else - if [ `command -v scala` ]; then - RUNNER="scala" - else - echo "SCALA_HOME is not set and scala is not in PATH" >&2 - exit 1 - fi - fi -else - if [ `command -v java` ]; then - RUNNER="java" - else - if [ -z "$JAVA_HOME" ]; then - echo "JAVA_HOME is not set" >&2 - exit 1 - fi - RUNNER="${JAVA_HOME}/bin/java" - fi - if [ -z "$SCALA_LIBRARY_PATH" ]; then - if [ -z "$SCALA_HOME" ]; then - echo "SCALA_HOME is not set" >&2 - exit 1 - fi - SCALA_LIBRARY_PATH="$SCALA_HOME/lib" - fi -fi - # Figure out how much memory to use per executor and set it as an environment # variable so that our process sees it and can report it to Mesos if [ -z "$SPARK_MEM" ] ; then @@ -93,64 +61,28 @@ if [ -e $FWDIR/conf/java-opts ] ; then fi export JAVA_OPTS -CORE_DIR="$FWDIR/core" -REPL_DIR="$FWDIR/repl" -REPL_BIN_DIR="$FWDIR/repl-bin" -EXAMPLES_DIR="$FWDIR/examples" -BAGEL_DIR="$FWDIR/bagel" -STREAMING_DIR="$FWDIR/streaming" -PYSPARK_DIR="$FWDIR/python" +# Check if this is a binary distribution or source distribution +# and build up the classpath appropriately +if [ -f "$FWDIR/RELEASE" ]; then + echo "This is a binary distribution" -# Exit if the user hasn't compiled Spark -if [ ! -e "$CORE_DIR/target" ]; then - echo "Failed to find Spark classes in $CORE_DIR/target" >&2 - echo "You need to compile Spark before running this program" >&2 - exit 1 -fi + if [ `command -v java` ]; then + RUNNER="java" + else + if [ -z "$JAVA_HOME" ]; then + echo "JAVA_HOME is not set" >&2 + exit 1 + fi + RUNNER="${JAVA_HOME}/bin/java" + fi -if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then - echo "Failed to find Spark classes in $REPL_DIR/target" >&2 - echo "You need to compile Spark repl module before running this program" >&2 - exit 1 -fi + CLASSPATH="$SPARK_CLASSPATH:$FWDIR/jars/*" -# Build up classpath -CLASSPATH="$SPARK_CLASSPATH" -CLASSPATH="$CLASSPATH:$FWDIR/conf" -CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" -if [ -n "$SPARK_TESTING" ] ; then - CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" -fi -CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" -CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar -if [ -e "$FWDIR/lib_managed" ]; then - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" -fi -CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" -if [ -e $REPL_BIN_DIR/target ]; then - for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do - CLASSPATH="$CLASSPATH:$jar" - done -fi -CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" -for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do - CLASSPATH="$CLASSPATH:$jar" -done - -# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack -# to avoid the -sources and -doc packages that are built by publish-local. -if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then - # Use the JAR from the SBT build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` -fi -if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then - # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` + # The JVM doesn't read JAVA_OPTS by default so we need to pass it in + EXTRA_ARGS="$JAVA_OPTS" +else + echo "This is a source distribution" + . "$FWDIR/set-dev-classpath.sh" fi # Add hadoop conf dir - else FileSystem.*, etc fail ! @@ -163,22 +95,5 @@ if [ "x" != "x$YARN_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" fi - -# Figure out whether to run our class with java or with the scala launcher. -# In most cases, we'd prefer to execute our process with java because scala -# creates a shell script as the parent of its Java process, which makes it -# hard to kill the child with stuff like Process.destroy(). However, for -# the Spark shell, the wrapper is necessary to properly reset the terminal -# when we exit, so we allow it to set a variable to launch with scala. -if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then - EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS -else - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" - # The JVM doesn't read JAVA_OPTS by default so we need to pass it in - EXTRA_ARGS="$JAVA_OPTS" -fi - export CLASSPATH # Needed for spark-shell exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" diff --git a/set-dev-classpath.sh b/set-dev-classpath.sh new file mode 100644 index 0000000000..4d09bd4416 --- /dev/null +++ b/set-dev-classpath.sh @@ -0,0 +1,112 @@ +# A BASH script to set the classpath for running Spark out of the developer/github tree + +SCALA_VERSION=2.9.3 + +# Figure out where the Scala framework is installed +FWDIR="$(cd `dirname $0`; pwd)" + +if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then + if [ "$SCALA_HOME" ]; then + RUNNER="${SCALA_HOME}/bin/scala" + else + if [ `command -v scala` ]; then + RUNNER="scala" + else + echo "SCALA_HOME is not set and scala is not in PATH" >&2 + exit 1 + fi + fi +else + if [ `command -v java` ]; then + RUNNER="java" + else + if [ -z "$JAVA_HOME" ]; then + echo "JAVA_HOME is not set" >&2 + exit 1 + fi + RUNNER="${JAVA_HOME}/bin/java" + fi + if [ -z "$SCALA_LIBRARY_PATH" ]; then + if [ -z "$SCALA_HOME" ]; then + echo "SCALA_HOME is not set" >&2 + exit 1 + fi + SCALA_LIBRARY_PATH="$SCALA_HOME/lib" + fi +fi + +CORE_DIR="$FWDIR/core" +REPL_DIR="$FWDIR/repl" +REPL_BIN_DIR="$FWDIR/repl-bin" +EXAMPLES_DIR="$FWDIR/examples" +BAGEL_DIR="$FWDIR/bagel" +STREAMING_DIR="$FWDIR/streaming" +PYSPARK_DIR="$FWDIR/python" + +# Exit if the user hasn't compiled Spark +if [ ! -e "$CORE_DIR/target" ]; then + echo "Failed to find Spark classes in $CORE_DIR/target" >&2 + echo "You need to compile Spark before running this program" >&2 + exit 1 +fi + +if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then + echo "Failed to find Spark classes in $REPL_DIR/target" >&2 + echo "You need to compile Spark repl module before running this program" >&2 + exit 1 +fi + +# Build up classpath +CLASSPATH="$SPARK_CLASSPATH" +CLASSPATH="$CLASSPATH:$FWDIR/conf" +CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" +if [ -n "$SPARK_TESTING" ] ; then + CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" +fi +CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" +CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar +if [ -e "$FWDIR/lib_managed" ]; then + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" +fi +CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" +if [ -e $REPL_BIN_DIR/target ]; then + for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do + CLASSPATH="$CLASSPATH:$jar" + done +fi +CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do + CLASSPATH="$CLASSPATH:$jar" +done + +# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack +# to avoid the -sources and -doc packages that are built by publish-local. +if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` +fi +if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then + # Use the JAR from the Maven build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` +fi + +# Figure out whether to run our class with java or with the scala launcher. +# In most cases, we'd prefer to execute our process with java because scala +# creates a shell script as the parent of its Java process, which makes it +# hard to kill the child with stuff like Process.destroy(). However, for +# the Spark shell, the wrapper is necessary to properly reset the terminal +# when we exit, so we allow it to set a variable to launch with scala. +if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then + EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS +else + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" + # The JVM doesn't read JAVA_OPTS by default so we need to pass it in + EXTRA_ARGS="$JAVA_OPTS" +fi From 42157027f247035e9ae41efe899e27c0942f5cd8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 16:25:05 -0700 Subject: [PATCH 046/419] A few bug fixes and a unit test --- core/src/main/scala/spark/SparkContext.scala | 1 + .../main/scala/spark/deploy/master/Master.scala | 9 +++++---- .../spark/deploy/master/ui/MasterWebUI.scala | 10 ++++++++-- .../main/scala/spark/deploy/worker/Worker.scala | 9 +++------ .../spark/deploy/worker/ui/WorkerWebUI.scala | 10 ++++++++-- core/src/main/scala/spark/ui/JettyUtils.scala | 7 +++++-- core/src/main/scala/spark/ui/SparkUI.scala | 13 ++++++++++--- core/src/test/resources/log4j.properties | 1 + core/src/test/scala/spark/DistributedSuite.scala | 2 ++ core/src/test/scala/spark/ui/UISuite.scala | 15 +++++++++++++++ 10 files changed, 58 insertions(+), 19 deletions(-) create mode 100644 core/src/test/scala/spark/ui/UISuite.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1f420d73f7..500d25efdd 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -578,6 +578,7 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { + ui.stop() // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3eca522c7e..2a49dfb486 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -36,6 +36,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act var firstApp: Option[ApplicationInfo] = None + val webUi = new MasterWebUI(self) + Utils.checkHost(host, "Expected hostname") val masterPublicAddress = { @@ -52,13 +54,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logInfo("Starting Spark master at spark://" + host + ":" + port) // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) - startWebUi() + webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) } - def startWebUi() { - val webUi = new MasterWebUI(self) - webUi.start() + override def postStop() { + webUi.stop() } override def receive = { diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index 5fd17f10c6..065b62a4b2 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -5,7 +5,7 @@ import akka.util.Duration import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, Utils} import spark.ui.JettyUtils @@ -21,13 +21,15 @@ class MasterWebUI(val master: ActorRef) extends Logging { val host = Utils.localHostName() val port = Option(System.getProperty("master.ui.port")) .getOrElse(MasterWebUI.DEFAULT_PORT).toInt + var server: Option[Server] = None val applicationPage = new ApplicationPage(this) val indexPage = new IndexPage(this) def start() { try { - val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + server = Some(srv) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => @@ -42,6 +44,10 @@ class MasterWebUI(val master: ActorRef) extends Logging { ("/app", (request: HttpServletRequest) => applicationPage.render(request)), ("*", (request: HttpServletRequest) => indexPage.render(request)) ) + + def stop() { + server.foreach(_.stop()) + } } object MasterWebUI { diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 690bdfe128..28553b6c02 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -45,6 +45,7 @@ private[spark] class Worker( val envVar = System.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } + val webUi = new WorkerWebUI(self, workDir) var coresUsed = 0 var memoryUsed = 0 @@ -77,7 +78,7 @@ private[spark] class Worker( logInfo("Spark home: " + sparkHome) createWorkDir() connectToMaster() - startWebUi() + webUi.start() } def connectToMaster() { @@ -88,11 +89,6 @@ private[spark] class Worker( context.watch(master) // Doesn't work with remote actors, but useful for testing } - def startWebUi() { - val webUi = new WorkerWebUI(self, workDir) - webUi.start() - } - override def receive = { case RegisteredWorker(url) => masterWebUiUrl = url @@ -163,6 +159,7 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) + webUi.stop() } } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index abfc847527..ee3889192d 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -7,7 +7,7 @@ import java.io.File import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.server.{Handler, Server} import scala.io.Source @@ -25,6 +25,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { val host = Utils.localHostName() val port = Option(System.getProperty("wroker.ui.port")) .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt + var server: Option[Server] = None val indexPage = new IndexPage(this) @@ -37,7 +38,8 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { def start() { try { - val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + server = Some(srv) logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => @@ -56,6 +58,10 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { source.close() lines } + + def stop() { + server.foreach(_.stop()) + } } object WorkerWebUI { diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index b5270e6062..85d6a7e867 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -12,7 +12,8 @@ import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHa import scala.util.{Try, Success, Failure} import scala.xml.Node -import spark.{Utils, SparkContext, Logging} +import spark.{SparkContext, Logging} +import org.eclipse.jetty.util.log.Log /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { @@ -91,12 +92,14 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { - val server = new Server(port) + val server = new Server(currentPort) server.setHandler(handlerList) Try { server.start() } match { case s: Success[_] => (server, currentPort) case f: Failure[_] => + server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) + logInfo("Error was: " + f.toString) connect((currentPort + 1) % 65536) } } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 2d5a328015..487f005615 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -2,7 +2,7 @@ package spark.ui import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, SparkContext, Utils} import spark.ui.storage.BlockManagerUI @@ -17,6 +17,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val host = Utils.localHostName() val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None + var server: Option[Server] = None val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), @@ -26,11 +27,12 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val jobs = new JobProgressUI(sc) val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers + /** Bind the HTTP server which backs this web interface */ def bind() { - /** Start an HTTP server to run the Web interface */ try { - val (server, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) + val (srv, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) + server = Some(srv) boundPort = Some(usedPort) } catch { case e: Exception => @@ -38,6 +40,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { System.exit(1) } } + /** Initialize all components of the server */ def start() { // NOTE: This is decoupled from bind() because of the following dependency cycle: @@ -47,6 +50,10 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { jobs.start() } + def stop() { + server.foreach(_.stop()) + } + private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") } diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 6ec89c0184..d05cf3dec1 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -8,3 +8,4 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 0866fb47b3..0024ede828 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -10,6 +10,7 @@ import org.scalatest.time.{Span, Millis} import org.scalacheck.Arbitrary._ import org.scalacheck.Gen import org.scalacheck.Prop._ +import org.eclipse.jetty.server.{Server, Request, Handler} import com.google.common.io.Files @@ -17,6 +18,7 @@ import scala.collection.mutable.ArrayBuffer import SparkContext._ import storage.{GetBlock, BlockManagerWorker, StorageLevel} +import ui.JettyUtils class NotSerializableClass diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala new file mode 100644 index 0000000000..6766b158f6 --- /dev/null +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -0,0 +1,15 @@ +package spark.ui + +import org.scalatest.FunSuite +import org.eclipse.jetty.server.Server + +class UISuite extends FunSuite { + test("jetty port increases under contention") { + val startPort = 33333 + val server = new Server(startPort) + server.start() + val (jettyServer, boundPort) = JettyUtils.startJettyServer("localhost", startPort, Seq()) + assert(boundPort === startPort + 1) + } + +} From f7389330c35a6372c4c7b455b96b8498b9a9da27 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 16:51:52 -0700 Subject: [PATCH 047/419] Allowing for requested port on construction --- .../main/scala/spark/deploy/master/Master.scala | 2 +- .../scala/spark/deploy/master/ui/MasterWebUI.scala | 11 +++++++---- .../main/scala/spark/deploy/worker/Worker.scala | 6 +++--- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 14 +++++++++----- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- 5 files changed, 21 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 2a49dfb486..127be9e7d4 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -71,7 +71,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } else { addWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) context.watch(sender) // This doesn't work with remote actors but helps for testing - sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUiPort) + sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort) schedule() } } diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index 065b62a4b2..50e93cd00f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -15,21 +15,24 @@ import spark.ui.JettyUtils._ * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: ActorRef) extends Logging { +class MasterWebUI(val master: ActorRef, requestedPort: Option[Int]) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() - val port = Option(System.getProperty("master.ui.port")) - .getOrElse(MasterWebUI.DEFAULT_PORT).toInt + val port = requestedPort.getOrElse( + System.getProperty("master.ui.port", MasterWebUI.DEFAULT_PORT).toInt) + var server: Option[Server] = None + var boundPort: Option[Int] = None val applicationPage = new ApplicationPage(this) val indexPage = new IndexPage(this) def start() { try { - val (srv, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) server = Some(srv) + boundPort = Some(bPort) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 28553b6c02..b6a26245fc 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -45,7 +45,7 @@ private[spark] class Worker( val envVar = System.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - val webUi = new WorkerWebUI(self, workDir) + val webUi = new WorkerWebUI(self, workDir, Some(webUiPort)) var coresUsed = 0 var memoryUsed = 0 @@ -77,14 +77,14 @@ private[spark] class Worker( sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() - connectToMaster() webUi.start() + connectToMaster() } def connectToMaster() { logInfo("Connecting to master " + masterUrl) master = context.actorFor(Master.toAkkaUrl(masterUrl)) - master ! RegisterWorker(workerId, host, port, cores, memory, webUiPort, publicAddress) + master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index ee3889192d..2615458dee 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -19,13 +19,16 @@ import spark.ui.JettyUtils._ * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { +class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option[Int] = None) + extends Logging { implicit val timeout = Timeout( Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) val host = Utils.localHostName() - val port = Option(System.getProperty("wroker.ui.port")) - .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt + val port = requestedPort.getOrElse( + System.getProperty("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + var server: Option[Server] = None + var boundPort: Option[Int] = None val indexPage = new IndexPage(this) @@ -38,9 +41,10 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { def start() { try { - val (srv, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) server = Some(srv) - logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) + boundPort = Some(bPort) + logInfo("Started Worker web UI at http://%s:%d".format(host, bPort)) } catch { case e: Exception => logError("Failed to create Worker JettyUtils", e) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 85d6a7e867..1f665cbb42 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -95,7 +95,7 @@ private[spark] object JettyUtils extends Logging { val server = new Server(currentPort) server.setHandler(handlerList) Try { server.start() } match { - case s: Success[_] => (server, currentPort) + case s: Success[_] => (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) From 81df20e5b44407c8c321471be2faee4bf6853fd6 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 17:05:37 -0700 Subject: [PATCH 048/419] Script to create binary distribution for Spark --- make-distribution.sh | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100755 make-distribution.sh diff --git a/make-distribution.sh b/make-distribution.sh new file mode 100755 index 0000000000..34f6d4a36b --- /dev/null +++ b/make-distribution.sh @@ -0,0 +1,30 @@ +#!/bin/bash +# +# Script to create a binary distribution for easy deploys of Spark. +# The distribution directory defaults to dist/ but can be overridden below. +# The distribution contains fat (assembly) jars that include the Scala library, +# so it is completely self contained. + +# Figure out where the Spark framework is installed +FWDIR="$(cd `dirname $0`; pwd)" +DISTDIR="$FWDIR/dist" + +# Get version from SBT +VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2) +echo "Making distribution for Spark $VERSION in $DISTDIR..." + +# Build fat JAR +$FWDIR/sbt/sbt "repl/assembly" + +# Make directories +rm -rf "$DISTDIR" +mkdir -p "$DISTDIR/jars" +echo "$VERSION" >$DISTDIR/RELEASE + +# Copy jars +cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" + +# Copy other things +cp -r "$FWDIR/bin" "$DISTDIR" +cp -r "$FWDIR/conf" "$DISTDIR" +cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" \ No newline at end of file From d66bd6f8851e87c8f35cca86ee45e26a8f547040 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 17:08:58 -0700 Subject: [PATCH 049/419] Adding another unit test to Web UI suite --- .../spark/deploy/master/ui/MasterWebUI.scala | 4 ++-- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- core/src/test/scala/spark/ui/UISuite.scala | 18 ++++++++++++++++-- 3 files changed, 19 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index 50e93cd00f..dfefd6571b 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -15,7 +15,7 @@ import spark.ui.JettyUtils._ * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: ActorRef, requestedPort: Option[Int]) extends Logging { +class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() @@ -33,7 +33,7 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int]) extends Logg val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) server = Some(srv) boundPort = Some(bPort) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) + logInfo("Started Master web UI at http://%s:%d".format(host, boundPort.get)) } catch { case e: Exception => logError("Failed to create Master JettyUtils", e) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 1f665cbb42..45c0805b22 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -62,7 +62,7 @@ private[spark] object JettyUtils extends Logging { } } - /** Creates a handler for serving files from a static directory. */ + /** Creates a handler for serving files from a static directory */ def createStaticHandler(resourceBase: String): ResourceHandler = { val staticHandler = new ResourceHandler Option(getClass.getClassLoader.getResource(resourceBase)) match { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 6766b158f6..7df7356a6e 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -2,14 +2,28 @@ package spark.ui import org.scalatest.FunSuite import org.eclipse.jetty.server.Server +import util.{Try, Success, Failure} +import java.net.ServerSocket class UISuite extends FunSuite { test("jetty port increases under contention") { val startPort = 33333 val server = new Server(startPort) server.start() - val (jettyServer, boundPort) = JettyUtils.startJettyServer("localhost", startPort, Seq()) - assert(boundPort === startPort + 1) + val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq()) + val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("localhost", startPort, Seq()) + + assert(boundPort1 === startPort + 1) + assert(boundPort2 === startPort + 2) } + test("jetty binds to port 0 correctly") { + val (jettyServer, boundPort) = JettyUtils.startJettyServer("localhost", 0, Seq()) + assert(jettyServer.getState === "STARTED") + assert(boundPort != 0) + Try {new ServerSocket(boundPort)} match { + case Success(s) => fail("Port %s doesn't seem used by jetty server".format(boundPort)) + case Failure(e) => + } + } } From cee05a174897af294f52cbda551da09cb869557e Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 23:55:09 -0700 Subject: [PATCH 050/419] Copy restore-TTY functions from Scala script so binary distros don't need 'scala' installed --- spark-shell | 39 ++++++++++++++++++++++++++++++++++++--- 1 file changed, 36 insertions(+), 3 deletions(-) diff --git a/spark-shell b/spark-shell index 574ae2104d..afbb7a9a8e 100755 --- a/spark-shell +++ b/spark-shell @@ -1,4 +1,37 @@ -#!/bin/sh +#!/bin/bash --posix FWDIR="`dirname $0`" -export SPARK_LAUNCH_WITH_SCALA=1 -exec $FWDIR/run spark.repl.Main "$@" + +# Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in +# binary distribution of Spark where Scala is not installed +exit_status=127 +saved_stty="" + +# restore stty settings (echo in particular) +function restoreSttySettings() { + stty $saved_stty + saved_stty="" +} + +function onExit() { + if [[ "$saved_stty" != "" ]]; then + restoreSttySettings + fi + exit $exit_status +} + +# to reenable echo if we are interrupted before completing. +trap onExit INT + +# save terminal settings +saved_stty=$(stty -g 2>/dev/null) +# clear on error so we don't later try to restore them +if [[ ! $? ]]; then + saved_stty="" +fi + +$FWDIR/run spark.repl.Main "$@" + +# record the exit status lest it be overwritten: +# then reenable echo and propagate the code. +exit_status=$? +onExit From c3d11d0d57739ec3f08783f71bf4d0efdec3d627 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 00:02:00 -0700 Subject: [PATCH 051/419] Get rid of debugging statements --- run | 3 --- 1 file changed, 3 deletions(-) diff --git a/run b/run index 30a2885a4d..646d12c1eb 100755 --- a/run +++ b/run @@ -64,8 +64,6 @@ export JAVA_OPTS # Check if this is a binary distribution or source distribution # and build up the classpath appropriately if [ -f "$FWDIR/RELEASE" ]; then - echo "This is a binary distribution" - if [ `command -v java` ]; then RUNNER="java" else @@ -81,7 +79,6 @@ if [ -f "$FWDIR/RELEASE" ]; then # The JVM doesn't read JAVA_OPTS by default so we need to pass it in EXTRA_ARGS="$JAVA_OPTS" else - echo "This is a source distribution" . "$FWDIR/set-dev-classpath.sh" fi From 243d71cb066d888f3d1a9c613859522438b69ba8 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 00:15:58 -0700 Subject: [PATCH 052/419] Add deploy/testing procedure --- make-distribution.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/make-distribution.sh b/make-distribution.sh index 34f6d4a36b..855475864d 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -4,6 +4,14 @@ # The distribution directory defaults to dist/ but can be overridden below. # The distribution contains fat (assembly) jars that include the Scala library, # so it is completely self contained. +# It does not contain source or *.class files. +# +# Recommended deploy/testing procedure (standalone mode): +# 1) Rsync / deploy the dist/ dir to one host +# 2) cd to deploy dir; ./bin/start-master.sh +# 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. +# 4) ./bin/start-slave.sh 1 <> +# 5) MASTER="spark://my-master-ip:7077" ./spark-shell # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" From 982a686ff858399067de960fa62cc80f60c6fa32 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 11:14:21 -0700 Subject: [PATCH 053/419] Add -m and -c options to spark-shell for convenience --- spark-shell | 29 ++++++++++++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/spark-shell b/spark-shell index afbb7a9a8e..ea67a3e6b8 100755 --- a/spark-shell +++ b/spark-shell @@ -1,6 +1,33 @@ #!/bin/bash --posix +# +# Shell script for starting the Spark Shell REPL +# Options: +# -m Set MASTER to spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT +# -c Set the number of cores for REPL to use +# FWDIR="`dirname $0`" +for o in "$@"; do + if [ "$1" = "-m" -o "$1" = "--master" ]; then + shift + if [ -e "$FWDIR/conf/spark-env.sh" ]; then + . "$FWDIR/conf/spark-env.sh" + fi + if [ -z "$MASTER" ]; then + MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" + fi + export MASTER + fi + + if [ "$1" = "-c" -o "$1" = "--cores" ]; then + shift + if [ -n "$1" ]; then + OPTIONS="-Dspark.cores.max=$1" + shift + fi + fi +done + # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed exit_status=127 @@ -29,7 +56,7 @@ if [[ ! $? ]]; then saved_stty="" fi -$FWDIR/run spark.repl.Main "$@" +$FWDIR/run $OPTIONS spark.repl.Main "$@" # record the exit status lest it be overwritten: # then reenable echo and propagate the code. From 366572edcab87701fd795ca0142ac9829b312d36 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Tue, 25 Jun 2013 22:59:34 +0100 Subject: [PATCH 054/419] Include a default OAuth implementation, and update examples and JavaStreamingContext --- .../examples/TwitterAlgebirdCMS.scala | 2 +- .../examples/TwitterAlgebirdHLL.scala | 2 +- .../examples/TwitterPopularTags.scala | 2 +- .../spark/streaming/StreamingContext.scala | 2 +- .../api/java/JavaStreamingContext.scala | 69 +++++++++++++------ .../dstream/TwitterInputDStream.scala | 32 ++++++++- 6 files changed, 81 insertions(+), 28 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index a9642100e3..548190309e 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -45,7 +45,7 @@ object TwitterAlgebirdCMS { val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index f3288bfb85..5a86c6318d 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -34,7 +34,7 @@ object TwitterAlgebirdHLL { val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 9d4494c6f2..076c3878c8 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -23,7 +23,7 @@ object TwitterPopularTags { val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters) + val stream = ssc.twitterStream(None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 05be6bd58a..0f36504c0d 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -368,7 +368,7 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects */ def twitterStream( - twitterAuth: Authorization, + twitterAuth: Option[Authorization] = None, filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): DStream[Status] = { diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 3d149a742c..85390ef57e 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -4,23 +4,18 @@ import spark.streaming._ import receivers.{ActorReceiver, ReceiverSupervisorStrategy} import spark.streaming.dstream._ import spark.storage.StorageLevel - import spark.api.java.function.{Function => JFunction, Function2 => JFunction2} import spark.api.java.{JavaSparkContext, JavaRDD} - import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} - import twitter4j.Status - import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe - import scala.collection.JavaConversions._ - import java.lang.{Long => JLong, Integer => JInt} import java.io.InputStream import java.util.{Map => JMap} +import twitter4j.auth.Authorization /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -315,46 +310,78 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password + * @param twitterAuth Twitter4J Authorization * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ def twitterStream( - username: String, - password: String, + twitterAuth: Authorization, filters: Array[String], storageLevel: StorageLevel ): JavaDStream[Status] = { - ssc.twitterStream(username, password, filters, storageLevel) + ssc.twitterStream(Some(twitterAuth), filters, storageLevel) + } + + /** + * Create a input stream that returns tweets received from Twitter using + * java.util.Preferences to store OAuth token. OAuth key and secret should + * be provided using system properties twitter4j.oauth.consumerKey and + * twitter4j.oauth.consumerSecret + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + filters: Array[String], + storageLevel: StorageLevel + ): JavaDStream[Status] = { + ssc.twitterStream(None, filters, storageLevel) } /** * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password + * @param twitterAuth Twitter4J Authorization * @param filters Set of filter strings to get only those tweets that match them */ def twitterStream( - username: String, - password: String, + twitterAuth: Authorization, filters: Array[String] ): JavaDStream[Status] = { - ssc.twitterStream(username, password, filters) + ssc.twitterStream(Some(twitterAuth), filters) + } + + /** + * Create a input stream that returns tweets received from Twitter using + * java.util.Preferences to store OAuth token. OAuth key and secret should + * be provided using system properties twitter4j.oauth.consumerKey and + * twitter4j.oauth.consumerSecret + * @param filters Set of filter strings to get only those tweets that match them + */ + def twitterStream( + filters: Array[String] + ): JavaDStream[Status] = { + ssc.twitterStream(None, filters) } /** * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password + * @param twitterAuth Twitter4J Authorization */ def twitterStream( - username: String, - password: String + twitterAuth: Authorization ): JavaDStream[Status] = { - ssc.twitterStream(username, password) + ssc.twitterStream(Some(twitterAuth)) } + /** + * Create a input stream that returns tweets received from Twitter using + * java.util.Preferences to store OAuth token. OAuth key and secret should + * be provided using system properties twitter4j.oauth.consumerKey and + * twitter4j.oauth.consumerSecret + */ + def twitterStream(): JavaDStream[Status] = { + ssc.twitterStream() + } + /** * Create an input stream with any arbitrary user implemented actor receiver. * @param props Props object defining creation of the actor diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index 0b01091a52..e0c654d385 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -3,27 +3,53 @@ package spark.streaming.dstream import spark._ import spark.streaming._ import storage.StorageLevel - import twitter4j._ import twitter4j.auth.BasicAuthorization import twitter4j.auth.Authorization +import java.util.prefs.Preferences +import twitter4j.conf.PropertyConfiguration +import twitter4j.auth.OAuthAuthorization +import twitter4j.auth.AccessToken /* A stream of Twitter statuses, potentially filtered by one or more keywords. * * @constructor create a new Twitter stream using the supplied username and password to authenticate. * An optional set of string filters can be used to restrict the set of tweets. The Twitter API is * such that this may return a sampled subset of all tweets during each interval. +* +* Includes a simple implementation of OAuth using consumer key and secret provided using system +* properties twitter4j.oauth.consumerKey and twitter4j.oauth.consumerSecret */ private[streaming] class TwitterInputDStream( @transient ssc_ : StreamingContext, - twitterAuth: Authorization, + twitterAuth: Option[Authorization], filters: Seq[String], storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { + lazy val createOAuthAuthorization: Authorization = { + val userRoot = Preferences.userRoot(); + val token = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN, null)) + val tokenSecret = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, null)) + val oAuth = new OAuthAuthorization(new PropertyConfiguration(System.getProperties())) + if (token.isEmpty || tokenSecret.isEmpty) { + val requestToken = oAuth.getOAuthRequestToken() + println("Authorize application using URL: "+requestToken.getAuthorizationURL()) + println("Enter PIN: ") + val pin = Console.readLine + val accessToken = if (pin.length() > 0) oAuth.getOAuthAccessToken(requestToken, pin) else oAuth.getOAuthAccessToken() + userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN, accessToken.getToken()) + userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, accessToken.getTokenSecret()) + userRoot.flush() + } else { + oAuth.setOAuthAccessToken(new AccessToken(token.get, tokenSecret.get)); + } + oAuth + } + override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(twitterAuth, filters, storageLevel) + new TwitterReceiver(if (twitterAuth.isEmpty) createOAuthAuthorization else twitterAuth.get, filters, storageLevel) } } From 8dd78f80b578751df164772a01772aad26540ac9 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 15:01:06 -0700 Subject: [PATCH 055/419] Manually merge spark/master commit d92d3f7938dec954ea31de232f50cafd4b644065 --- set-dev-classpath.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/set-dev-classpath.sh b/set-dev-classpath.sh index 4d09bd4416..d031c56baf 100644 --- a/set-dev-classpath.sh +++ b/set-dev-classpath.sh @@ -74,10 +74,15 @@ if [ -e "$FWDIR/lib_managed" ]; then CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" fi CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" + +# Add the shaded JAR for Maven builds if [ -e $REPL_BIN_DIR/target ]; then for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do CLASSPATH="$CLASSPATH:$jar" done + # The shaded JAR doesn't contain examples, so include those separately + EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + CLASSPATH+=":$EXAMPLES_JAR" fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do @@ -90,11 +95,12 @@ if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; # Use the JAR from the SBT build export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` fi -if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then +if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` fi + # Figure out whether to run our class with java or with the scala launcher. # In most cases, we'd prefer to execute our process with java because scala # creates a shell script as the parent of its Java process, which makes it From 176193b1e8acdbe2f1cfaed16b8f42f89e226f79 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Tue, 25 Jun 2013 23:06:15 +0100 Subject: [PATCH 056/419] Fix usage and parameter extraction --- .../spark/streaming/examples/TwitterAlgebirdCMS.scala | 7 +++---- .../spark/streaming/examples/TwitterAlgebirdHLL.scala | 7 +++---- .../spark/streaming/examples/TwitterPopularTags.scala | 7 +++---- 3 files changed, 9 insertions(+), 12 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index 548190309e..528778ed72 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -26,8 +26,8 @@ import spark.SparkContext._ */ object TwitterAlgebirdCMS { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterAlgebirdCMS " + + if (args.length < 1) { + System.err.println("Usage: TwitterAlgebirdCMS " + " [filter1] [filter2] ... [filter n]") System.exit(1) } @@ -40,8 +40,7 @@ object TwitterAlgebirdCMS { // K highest frequency elements to take val TOPK = 10 - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index 5a86c6318d..896e9fd8af 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -21,16 +21,15 @@ import spark.streaming.dstream.TwitterInputDStream */ object TwitterAlgebirdHLL { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterAlgebirdHLL " + + if (args.length < 1) { + System.err.println("Usage: TwitterAlgebirdHLL " + " [filter1] [filter2] ... [filter n]") System.exit(1) } /** Bit size parameter for HyperLogLog, trades off accuracy vs size */ val BIT_SIZE = 12 - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 076c3878c8..65f0b6d352 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -12,14 +12,13 @@ import spark.SparkContext._ */ object TwitterPopularTags { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterPopularTags " + + if (args.length < 1) { + System.err.println("Usage: TwitterPopularTags " + " [filter1] [filter2] ... [filter n]") System.exit(1) } - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) From 2cbaa0734b1a23e5d6ce1543a22358f436b5b1ef Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:44:55 -0700 Subject: [PATCH 057/419] Making all new classes package private --- core/src/main/scala/spark/deploy/JsonProtocol.scala | 2 +- .../main/scala/spark/deploy/master/ui/ApplicationPage.scala | 2 +- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala | 2 +- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- core/src/main/scala/spark/ui/storage/BlockManagerUI.scala | 3 +-- core/src/main/scala/spark/ui/storage/IndexPage.scala | 2 +- core/src/test/scala/spark/ui/UISuite.scala | 2 +- 13 files changed, 13 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index b4365d31e9..335aaeb769 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -4,7 +4,7 @@ import master.{ApplicationInfo, WorkerInfo} import net.liftweb.json.JsonDSL._ import worker.ExecutorRunner -object JsonProtocol { +private[spark] object JsonProtocol { def writeWorkerInfo(obj: WorkerInfo) = { ("id" -> obj.id) ~ ("host" -> obj.host) ~ diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 2a810b71d2..9cb9c94e9f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -14,7 +14,7 @@ import spark.deploy.{RequestMasterState, JsonProtocol, MasterState} import spark.deploy.master.ExecutorInfo import spark.ui.UIUtils -class ApplicationPage(parent: MasterWebUI) { +private[spark] class ApplicationPage(parent: MasterWebUI) { val master = parent.master implicit val timeout = parent.timeout diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index f833c59de8..3c3679a4b0 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -13,7 +13,7 @@ import spark.Utils import spark.ui.UIUtils import spark.deploy.master.{ApplicationInfo, WorkerInfo} -class IndexPage(parent: MasterWebUI) { +private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.master implicit val timeout = parent.timeout diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index dfefd6571b..0cfe600043 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -53,7 +53,7 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten } } -object MasterWebUI { +private[spark] object MasterWebUI { val STATIC_RESOURCE_DIR = "spark/webui/static" val DEFAULT_PORT = "8080" } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 9e7b2bfd92..c1ca09fc97 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -15,7 +15,7 @@ import spark.deploy.worker.ExecutorRunner import spark.Utils import spark.ui.UIUtils -class IndexPage(parent: WorkerWebUI) { +private[spark] class IndexPage(parent: WorkerWebUI) { val worker = parent.worker val timeout = parent.timeout diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 2615458dee..65bdba4735 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -68,7 +68,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } } -object WorkerWebUI { +private[spark] object WorkerWebUI { val STATIC_RESOURCE_DIR = "spark/webui/static" val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 45c0805b22..1f311bb75f 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -108,7 +108,7 @@ private[spark] object JettyUtils extends Logging { } /** Utility functions for generating XML pages with spark content. */ -object UIUtils { +private[spark] object UIUtils { /** Returns a page containing the supplied content and the spark web ui headers */ def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 487f005615..86b394c09f 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -57,7 +57,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") } -object SparkUI { +private[spark] object SparkUI { val DEFAULT_PORT = "33000" val STATIC_RESOURCE_DIR = "spark/webui/static" } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index c7ee9dc1c1..152e1a79b0 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -14,7 +14,7 @@ import spark.ui.UIUtils._ import spark.storage.StorageLevel /** Page showing list of all ongoing and recently finished stages */ -class IndexPage(parent: JobProgressUI) { +private[spark] class IndexPage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 65dbd389b1..03200861e0 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -13,7 +13,7 @@ import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics /** Page showing statistics and task list for a given stage */ -class StagePage(parent: JobProgressUI) { +private[spark] class StagePage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index 4f1928fac9..3d67888e2c 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -10,8 +10,7 @@ import spark.{Logging, SparkContext} import spark.ui.JettyUtils._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[spark] -class BlockManagerUI(val sc: SparkContext) extends Logging { +private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index cd7f6bd9f2..772c669a1a 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -9,7 +9,7 @@ import spark.Utils import spark.ui.UIUtils._ /** Page showing list of RDD's currently stored in the cluster */ -class IndexPage(parent: BlockManagerUI) { +private[spark] class IndexPage(parent: BlockManagerUI) { val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 7df7356a6e..6519bf77c6 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -5,7 +5,7 @@ import org.eclipse.jetty.server.Server import util.{Try, Success, Failure} import java.net.ServerSocket -class UISuite extends FunSuite { +private[spark] class UISuite extends FunSuite { test("jetty port increases under contention") { val startPort = 33333 val server = new Server(startPort) From b14ad509ba4c00d57bd4a52a7b70a18325064a7b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:46:51 -0700 Subject: [PATCH 058/419] Moving static ui package --- .../static/bootstrap-responsive.min.css | 0 .../spark/{webui => ui}/static/bootstrap.min.css | 0 .../spark/{webui => ui}/static/sorttable.js | 0 .../spark/{webui => ui}/static/spark_logo.png | Bin .../scala/spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- 7 files changed, 3 insertions(+), 3 deletions(-) rename core/src/main/resources/spark/{webui => ui}/static/bootstrap-responsive.min.css (100%) rename core/src/main/resources/spark/{webui => ui}/static/bootstrap.min.css (100%) rename core/src/main/resources/spark/{webui => ui}/static/sorttable.js (100%) rename core/src/main/resources/spark/{webui => ui}/static/spark_logo.png (100%) diff --git a/core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css b/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css similarity index 100% rename from core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css rename to core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css diff --git a/core/src/main/resources/spark/webui/static/bootstrap.min.css b/core/src/main/resources/spark/ui/static/bootstrap.min.css similarity index 100% rename from core/src/main/resources/spark/webui/static/bootstrap.min.css rename to core/src/main/resources/spark/ui/static/bootstrap.min.css diff --git a/core/src/main/resources/spark/webui/static/sorttable.js b/core/src/main/resources/spark/ui/static/sorttable.js similarity index 100% rename from core/src/main/resources/spark/webui/static/sorttable.js rename to core/src/main/resources/spark/ui/static/sorttable.js diff --git a/core/src/main/resources/spark/webui/static/spark_logo.png b/core/src/main/resources/spark/ui/static/spark_logo.png similarity index 100% rename from core/src/main/resources/spark/webui/static/spark_logo.png rename to core/src/main/resources/spark/ui/static/spark_logo.png diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index 0cfe600043..82bc6961e2 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -54,6 +54,6 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten } private[spark] object MasterWebUI { - val STATIC_RESOURCE_DIR = "spark/webui/static" + val STATIC_RESOURCE_DIR = "spark/ui/static" val DEFAULT_PORT = "8080" } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 65bdba4735..2e6279566c 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -69,6 +69,6 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } private[spark] object WorkerWebUI { - val STATIC_RESOURCE_DIR = "spark/webui/static" + val STATIC_RESOURCE_DIR = "spark/ui/static" val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 86b394c09f..b3bdc2c490 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -59,5 +59,5 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { private[spark] object SparkUI { val DEFAULT_PORT = "33000" - val STATIC_RESOURCE_DIR = "spark/webui/static" + val STATIC_RESOURCE_DIR = "spark/ui/static" } From 274193664ae13adbe5215f5dcc2be11e672557b6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:49:58 -0700 Subject: [PATCH 059/419] Bumping timeouts --- .../main/scala/spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 9cb9c94e9f..fb2c803a91 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -22,7 +22,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] - val state = Await.result(stateFuture, 3 seconds) + val state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) @@ -33,7 +33,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { def render(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] - val state = Await.result(stateFuture, 3 seconds) + val state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 3c3679a4b0..4892f258e1 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -20,7 +20,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] - val state = Await.result(stateFuture, 3 seconds) + val state = Await.result(stateFuture, 30 seconds) val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") val workers = state.workers.sortBy(_.id) diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index c1ca09fc97..e95a74a37f 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -21,13 +21,13 @@ private[spark] class IndexPage(parent: WorkerWebUI) { def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] - val workerState = Await.result(stateFuture, 3 seconds) + val workerState = Await.result(stateFuture, 30 seconds) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] - val workerState = Await.result(stateFuture, 3 seconds) + val workerState = Await.result(stateFuture, 30 seconds) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") val runningExecutorTable = From a59c15a37eba0b585ba49a1b1b2b309f71e56d02 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:54:57 -0700 Subject: [PATCH 060/419] Adding config option for retained stages --- core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 2 +- docs/configuration.md | 11 +++++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 027eadde3a..46ba1d31af 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -42,7 +42,7 @@ private[spark] class JobProgressUI(val sc: SparkContext) { private[spark] class JobProgressListener extends SparkListener { // How many stages to remember - val RETAINED_STAGES = 1000 + val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt val activeStages = HashSet[Stage]() val completedStages = ListBuffer[Stage]() diff --git a/docs/configuration.md b/docs/configuration.md index 17fdbf04d1..00a2dd7369 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -135,9 +135,16 @@ Apart from these, the following properties are also available, and may be useful spark.ui.port - (random) + 33000 - Port for your application's dashboard, which shows memory usage of each RDD. + Port for your application's dashboard, which shows memory and workload data + + + + spark.ui.retained_stages + 1000 + + How many stages the Spark UI remembers before garbage collecting. From ee692482a609b5d42f7270b020d6f38fb5e466fd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 09:07:32 -0700 Subject: [PATCH 061/419] One more private class --- core/src/main/scala/spark/ui/storage/RDDPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 73f151959e..dd9fd2e4c5 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -10,7 +10,7 @@ import spark.Utils import spark.storage.BlockManagerMasterActor.BlockStatus /** Page showing storage details for a given RDD */ -class RDDPage(parent: BlockManagerUI) { +private[spark] class RDDPage(parent: BlockManagerUI) { val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { From 92a4c2a5f6946bfae2136c52a22899db196f5799 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 12:33:06 -0700 Subject: [PATCH 062/419] Fixing bug in local scheduler time recording --- .../main/scala/spark/scheduler/local/LocalScheduler.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 93d4318b29..9d375e1db8 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -153,10 +153,10 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: // this adds a bit of unnecessary overhead but matches how the Mesos Executor works. val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes) updateDependencies(taskFiles, taskJars) // Download any files added with addFile - val deserStart = System.currentTimeMillis() + val taskStart = System.currentTimeMillis() val deserializedTask = ser.deserialize[Task[_]]( taskBytes, Thread.currentThread.getContextClassLoader) - val deserTime = System.currentTimeMillis() - deserStart + val deserTime = System.currentTimeMillis() - taskStart // Run it val result: Any = deserializedTask.run(taskId) @@ -170,8 +170,9 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val resultToReturn = ser.deserialize[Any](serResult) val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( ser.serialize(Accumulators.values)) + val serviceTime = System.currentTimeMillis() - taskStart logInfo("Finished " + taskId) - deserializedTask.metrics.get.executorRunTime = deserTime.toInt//info.duration.toInt //close enough + deserializedTask.metrics.get.executorRunTime = serviceTime.toInt deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt val taskResult = new TaskResult(result, accumUpdates, deserializedTask.metrics.getOrElse(null)) From 362d996c81e2b3275a61df2266372ed470ea4d88 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 13:02:47 -0700 Subject: [PATCH 063/419] Handful of changes based on matei's review - Avoid exception when no tasks have finished for a stage - Adding DOCTYPE so css renders properly - Adding progress slider --- core/src/main/scala/spark/Utils.scala | 20 ++++++++++++++++ core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- .../main/scala/spark/ui/jobs/IndexPage.scala | 23 ++++++++++++++++--- .../scala/spark/ui/jobs/JobProgressUI.scala | 3 ++- .../main/scala/spark/ui/jobs/StagePage.scala | 14 +++++++++-- core/src/test/scala/spark/ui/UISuite.scala | 23 ++++++++++++++++--- .../main/scala/spark/streaming/Duration.scala | 6 ++++- 7 files changed, 80 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index f3621c6bee..6966ee9ee9 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -485,6 +485,26 @@ private object Utils extends Logging { "%.1f %s".formatLocal(Locale.US, value, unit) } + /** + * Returns a human-readable string representing a duration such as "35ms" + */ + def msDurationToString(ms: Long): String = { + val second = 1000 + val minute = 60 * second + val hour = 60 * minute + + ms match { + case t if t < second => + "%dms".format(t) + case t if t < minute => + "%d.%03ds".format(t / second, t % second) + case t if t < hour => + "%d:%02d".format(t / minute, (t % minute) / second) + case t => + "%d:%02d:%02d".format(t / hour, t % hour / minute, (t % hour) % minute / second) + } + } + /** * Convert a memory quantity in megabytes to a human-readable string such as "4.0 MB". */ diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 1f311bb75f..fde3606740 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -26,7 +26,7 @@ private[spark] object JettyUtils extends Logging { createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = - createHandler(responder, "text/html") + createHandler(responder, "text/html", (in: Seq[Node]) => "" + in.toString) implicit def textResponderToHandler(responder: Responder[String]): Handler = createHandler(responder, "text/plain") diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 152e1a79b0..9b34cdd27f 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -19,8 +19,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { val dateFmt = parent.dateFmt def render(request: HttpServletRequest): Seq[Node] = { - val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total", - "Shuffle Activity", "Stored RDD") + val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Progress", + "Tasks: Complete/Total", "Shuffle Activity", "Stored RDD") val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq @@ -43,6 +43,20 @@ private[spark] class IndexPage(parent: JobProgressUI) { } } + def makeSlider(completed: Int, total: Int): Seq[Node] = { + val width=130 + val height=15 + val completeWidth = (completed.toDouble / total) * width + + + + + + } + + def stageRow(showLink: Boolean = true)(s: Stage): Seq[Node] = { val submissionTime = s.submissionTime match { case Some(t) => dateFmt.format(new Date(t)) @@ -55,6 +69,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { case (false, true) => "Write" case _ => "" } + val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) + val totalTasks = s.numPartitions {if (showLink) {{s.id}} @@ -63,7 +79,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} + {makeSlider(completedTasks, totalTasks)} + {completedTasks} / {totalTasks} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) case _ => diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 46ba1d31af..a18bf0f81e 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -17,6 +17,7 @@ import spark.scheduler._ import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics import spark.Success +import spark.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { @@ -32,7 +33,7 @@ private[spark] class JobProgressUI(val sc: SparkContext) { sc.addSparkListener(listener) } - def formatDuration(ms: Long) = Duration(ms, "milliseconds").printHMS + def formatDuration(ms: Long) = Utils.msDurationToString(ms) def getHandlers = Seq[(String, Handler)]( ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 03200861e0..518f48cb81 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -19,20 +19,30 @@ private[spark] class StagePage(parent: JobProgressUI) { def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt + + if (!listener.stageToTaskInfos.contains(stageId)) { + val content = +
    +

    Summary Metrics

    No tasks have finished yet +

    Tasks

    No tasks have finished yet +
    + return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId)) + } + val tasks = listener.stageToTaskInfos(stageId) val shuffleRead = listener.hasShuffleRead(stageId) val shuffleWrite = listener.hasShuffleWrite(stageId) val taskHeaders: Seq[String] = - Seq("Task ID", "Service Time", "Locality Level", "Worker", "Launch Time") ++ + Seq("Task ID", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ {if (shuffleWrite) Seq("Shuffle Write") else Nil} val taskTable = listingTable(taskHeaders, taskRow, tasks) val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} - val serviceQuantiles = "Service Time" +: Distribution(serviceTimes).get.getQuantiles().map( + val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( ms => parent.formatDuration(ms.toLong)) def getQuantileCols(data: Seq[Double]) = diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 6519bf77c6..127ab5ebc2 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -2,10 +2,11 @@ package spark.ui import org.scalatest.FunSuite import org.eclipse.jetty.server.Server -import util.{Try, Success, Failure} import java.net.ServerSocket +import scala.util.{Failure, Success, Try} +import spark.Utils -private[spark] class UISuite extends FunSuite { +class UISuite extends FunSuite { test("jetty port increases under contention") { val startPort = 33333 val server = new Server(startPort) @@ -23,7 +24,23 @@ private[spark] class UISuite extends FunSuite { assert(boundPort != 0) Try {new ServerSocket(boundPort)} match { case Success(s) => fail("Port %s doesn't seem used by jetty server".format(boundPort)) - case Failure(e) => + case Failure (e) => } } + + test("string formatting of time durations") { + val second = 1000 + val minute = second * 60 + val hour = minute * 60 + def str = Utils.msDurationToString(_) + + assert(str(123) === "123ms") + assert(str(second) === "1.000s") + assert(str(second + 452) === "1.452s") + assert(str(hour) === "1:00:00") + assert(str(minute) === "1:00") + assert(str(minute + 4 * second + 34) === "1:04") + assert(str(10 * hour + minute + 4 * second) === "10:01:04") + assert(str(10 * hour + 59 * minute + 59 * second + 999) === "10:59:59") + } } diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala index ee26206e24..c2135195d8 100644 --- a/streaming/src/main/scala/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/spark/streaming/Duration.scala @@ -1,5 +1,7 @@ package spark.streaming +import spark.Utils + case class Duration (private val millis: Long) { def < (that: Duration): Boolean = (this.millis < that.millis) @@ -32,8 +34,10 @@ case class Duration (private val millis: Long) { def toFormattedString: String = millis.toString def milliseconds: Long = millis -} + def prettyPrint = Utils.msDurationToString(millis) + +} /** * Helper object that creates instance of [[spark.streaming.Duration]] representing From a55190d3143dae9a29650819621396175ef8a0dd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 16:36:10 -0700 Subject: [PATCH 064/419] Adding better tabs for UI headers. --- .../ui/static/bootstrap-responsive.min.css | 4 +- .../spark/ui/static/bootstrap.min.css | 4 +- .../main/resources/spark/ui/static/webui.css | 41 +++++++++ .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/master/ui/IndexPage.scala | 2 +- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/JettyUtils.scala | 85 ++++++++++++++----- .../main/scala/spark/ui/jobs/IndexPage.scala | 3 +- .../main/scala/spark/ui/jobs/StagePage.scala | 5 +- .../scala/spark/ui/storage/IndexPage.scala | 3 +- .../main/scala/spark/ui/storage/RDDPage.scala | 3 +- 11 files changed, 120 insertions(+), 34 deletions(-) create mode 100644 core/src/main/resources/spark/ui/static/webui.css diff --git a/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css b/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css index 1f55036a07..f4ede63f32 100644 --- a/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css +++ b/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css @@ -1,9 +1,9 @@ /*! - * Bootstrap Responsive v2.0.4 + * Bootstrap Responsive v2.3.2 * * Copyright 2012 Twitter, Inc * Licensed under the Apache License v2.0 * http://www.apache.org/licenses/LICENSE-2.0 * * Designed and built with all the love in the world @twitter by @mdo and @fat. - */.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:28px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.hidden{display:none;visibility:hidden}.visible-phone{display:none!important}.visible-tablet{display:none!important}.hidden-desktop{display:none!important}@media(max-width:767px){.visible-phone{display:inherit!important}.hidden-phone{display:none!important}.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}}@media(min-width:768px) and (max-width:979px){.visible-tablet{display:inherit!important}.hidden-tablet{display:none!important}.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}}@media(max-width:480px){.nav-collapse{-webkit-transform:translate3d(0,0,0)}.page-header h1 small{display:block;line-height:18px}input[type="checkbox"],input[type="radio"]{border:1px solid #ccc}.form-horizontal .control-group>label{float:none;width:auto;padding-top:0;text-align:left}.form-horizontal .controls{margin-left:0}.form-horizontal .control-list{padding-top:0}.form-horizontal .form-actions{padding-right:10px;padding-left:10px}.modal{position:absolute;top:10px;right:10px;left:10px;width:auto;margin:0}.modal.fade.in{top:auto}.modal-header .close{padding:10px;margin:-10px}.carousel-caption{position:static}}@media(max-width:767px){body{padding-right:20px;padding-left:20px}.navbar-fixed-top,.navbar-fixed-bottom{margin-right:-20px;margin-left:-20px}.container-fluid{padding:0}.dl-horizontal dt{float:none;width:auto;clear:none;text-align:left}.dl-horizontal dd{margin-left:0}.container{width:auto}.row-fluid{width:100%}.row,.thumbnails{margin-left:0}[class*="span"],.row-fluid [class*="span"]{display:block;float:none;width:auto;margin-left:0}.input-large,.input-xlarge,.input-xxlarge,input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input{display:block;width:100%;min-height:28px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.input-prepend input,.input-append input,.input-prepend input[class*="span"],.input-append input[class*="span"]{display:inline-block;width:auto}}@media(min-width:768px) and (max-width:979px){.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:724px}.span12{width:724px}.span11{width:662px}.span10{width:600px}.span9{width:538px}.span8{width:476px}.span7{width:414px}.span6{width:352px}.span5{width:290px}.span4{width:228px}.span3{width:166px}.span2{width:104px}.span1{width:42px}.offset12{margin-left:764px}.offset11{margin-left:702px}.offset10{margin-left:640px}.offset9{margin-left:578px}.offset8{margin-left:516px}.offset7{margin-left:454px}.offset6{margin-left:392px}.offset5{margin-left:330px}.offset4{margin-left:268px}.offset3{margin-left:206px}.offset2{margin-left:144px}.offset1{margin-left:82px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:28px;margin-left:2.762430939%;*margin-left:2.709239449638298%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:99.999999993%;*width:99.9468085036383%}.row-fluid .span11{width:91.436464082%;*width:91.38327259263829%}.row-fluid .span10{width:82.87292817100001%;*width:82.8197366816383%}.row-fluid .span9{width:74.30939226%;*width:74.25620077063829%}.row-fluid .span8{width:65.74585634900001%;*width:65.6926648596383%}.row-fluid .span7{width:57.182320438000005%;*width:57.129128948638304%}.row-fluid .span6{width:48.618784527%;*width:48.5655930376383%}.row-fluid .span5{width:40.055248616%;*width:40.0020571266383%}.row-fluid .span4{width:31.491712705%;*width:31.4385212156383%}.row-fluid .span3{width:22.928176794%;*width:22.874985304638297%}.row-fluid .span2{width:14.364640883%;*width:14.311449393638298%}.row-fluid .span1{width:5.801104972%;*width:5.747913482638298%}input,textarea,.uneditable-input{margin-left:0}input.span12,textarea.span12,.uneditable-input.span12{width:714px}input.span11,textarea.span11,.uneditable-input.span11{width:652px}input.span10,textarea.span10,.uneditable-input.span10{width:590px}input.span9,textarea.span9,.uneditable-input.span9{width:528px}input.span8,textarea.span8,.uneditable-input.span8{width:466px}input.span7,textarea.span7,.uneditable-input.span7{width:404px}input.span6,textarea.span6,.uneditable-input.span6{width:342px}input.span5,textarea.span5,.uneditable-input.span5{width:280px}input.span4,textarea.span4,.uneditable-input.span4{width:218px}input.span3,textarea.span3,.uneditable-input.span3{width:156px}input.span2,textarea.span2,.uneditable-input.span2{width:94px}input.span1,textarea.span1,.uneditable-input.span1{width:32px}}@media(min-width:1200px){.row{margin-left:-30px;*zoom:1}.row:before,.row:after{display:table;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:30px}.container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:1170px}.span12{width:1170px}.span11{width:1070px}.span10{width:970px}.span9{width:870px}.span8{width:770px}.span7{width:670px}.span6{width:570px}.span5{width:470px}.span4{width:370px}.span3{width:270px}.span2{width:170px}.span1{width:70px}.offset12{margin-left:1230px}.offset11{margin-left:1130px}.offset10{margin-left:1030px}.offset9{margin-left:930px}.offset8{margin-left:830px}.offset7{margin-left:730px}.offset6{margin-left:630px}.offset5{margin-left:530px}.offset4{margin-left:430px}.offset3{margin-left:330px}.offset2{margin-left:230px}.offset1{margin-left:130px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:28px;margin-left:2.564102564%;*margin-left:2.510911074638298%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.45299145300001%;*width:91.3997999636383%}.row-fluid .span10{width:82.905982906%;*width:82.8527914166383%}.row-fluid .span9{width:74.358974359%;*width:74.30578286963829%}.row-fluid .span8{width:65.81196581200001%;*width:65.7587743226383%}.row-fluid .span7{width:57.264957265%;*width:57.2117657756383%}.row-fluid .span6{width:48.717948718%;*width:48.6647572286383%}.row-fluid .span5{width:40.170940171000005%;*width:40.117748681638304%}.row-fluid .span4{width:31.623931624%;*width:31.5707401346383%}.row-fluid .span3{width:23.076923077%;*width:23.0237315876383%}.row-fluid .span2{width:14.529914530000001%;*width:14.4767230406383%}.row-fluid .span1{width:5.982905983%;*width:5.929714493638298%}input,textarea,.uneditable-input{margin-left:0}input.span12,textarea.span12,.uneditable-input.span12{width:1160px}input.span11,textarea.span11,.uneditable-input.span11{width:1060px}input.span10,textarea.span10,.uneditable-input.span10{width:960px}input.span9,textarea.span9,.uneditable-input.span9{width:860px}input.span8,textarea.span8,.uneditable-input.span8{width:760px}input.span7,textarea.span7,.uneditable-input.span7{width:660px}input.span6,textarea.span6,.uneditable-input.span6{width:560px}input.span5,textarea.span5,.uneditable-input.span5{width:460px}input.span4,textarea.span4,.uneditable-input.span4{width:360px}input.span3,textarea.span3,.uneditable-input.span3{width:260px}input.span2,textarea.span2,.uneditable-input.span2{width:160px}input.span1,textarea.span1,.uneditable-input.span1{width:60px}.thumbnails{margin-left:-30px}.thumbnails>li{margin-left:30px}.row-fluid .thumbnails{margin-left:0}}@media(max-width:979px){body{padding-top:0}.navbar-fixed-top,.navbar-fixed-bottom{position:static}.navbar-fixed-top{margin-bottom:18px}.navbar-fixed-bottom{margin-top:18px}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding:5px}.navbar .container{width:auto;padding:0}.navbar .brand{padding-right:10px;padding-left:10px;margin:0 0 0 -5px}.nav-collapse{clear:both}.nav-collapse .nav{float:none;margin:0 0 9px}.nav-collapse .nav>li{float:none}.nav-collapse .nav>li>a{margin-bottom:2px}.nav-collapse .nav>.divider-vertical{display:none}.nav-collapse .nav .nav-header{color:#999;text-shadow:none}.nav-collapse .nav>li>a,.nav-collapse .dropdown-menu a{padding:6px 15px;font-weight:bold;color:#999;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.nav-collapse .btn{padding:4px 10px 4px;font-weight:normal;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.nav-collapse .dropdown-menu li+li a{margin-bottom:2px}.nav-collapse .nav>li>a:hover,.nav-collapse .dropdown-menu a:hover{background-color:#222}.nav-collapse.in .btn-group{padding:0;margin-top:5px}.nav-collapse .dropdown-menu{position:static;top:auto;left:auto;display:block;float:none;max-width:none;padding:0;margin:0 15px;background-color:transparent;border:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.nav-collapse .dropdown-menu:before,.nav-collapse .dropdown-menu:after{display:none}.nav-collapse .dropdown-menu .divider{display:none}.nav-collapse .navbar-form,.nav-collapse .navbar-search{float:none;padding:9px 15px;margin:9px 0;border-top:1px solid #222;border-bottom:1px solid #222;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1)}.navbar .nav-collapse .nav.pull-right{float:none;margin-left:0}.nav-collapse,.nav-collapse.collapse{height:0;overflow:hidden}.navbar .btn-navbar{display:block}.navbar-static .navbar-inner{padding-right:10px;padding-left:10px}}@media(min-width:980px){.nav-collapse.collapse{height:auto!important;overflow:visible!important}} + */.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}@-ms-viewport{width:device-width}.hidden{display:none;visibility:hidden}.visible-phone{display:none!important}.visible-tablet{display:none!important}.hidden-desktop{display:none!important}.visible-desktop{display:inherit!important}@media(min-width:768px) and (max-width:979px){.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}.visible-tablet{display:inherit!important}.hidden-tablet{display:none!important}}@media(max-width:767px){.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}.visible-phone{display:inherit!important}.hidden-phone{display:none!important}}.visible-print{display:none!important}@media print{.visible-print{display:inherit!important}.hidden-print{display:none!important}}@media(min-width:1200px){.row{margin-left:-30px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:30px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:1170px}.span12{width:1170px}.span11{width:1070px}.span10{width:970px}.span9{width:870px}.span8{width:770px}.span7{width:670px}.span6{width:570px}.span5{width:470px}.span4{width:370px}.span3{width:270px}.span2{width:170px}.span1{width:70px}.offset12{margin-left:1230px}.offset11{margin-left:1130px}.offset10{margin-left:1030px}.offset9{margin-left:930px}.offset8{margin-left:830px}.offset7{margin-left:730px}.offset6{margin-left:630px}.offset5{margin-left:530px}.offset4{margin-left:430px}.offset3{margin-left:330px}.offset2{margin-left:230px}.offset1{margin-left:130px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.564102564102564%;*margin-left:2.5109110747408616%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.564102564102564%}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.45299145299145%;*width:91.39979996362975%}.row-fluid .span10{width:82.90598290598291%;*width:82.8527914166212%}.row-fluid .span9{width:74.35897435897436%;*width:74.30578286961266%}.row-fluid .span8{width:65.81196581196582%;*width:65.75877432260411%}.row-fluid .span7{width:57.26495726495726%;*width:57.21176577559556%}.row-fluid .span6{width:48.717948717948715%;*width:48.664757228587014%}.row-fluid .span5{width:40.17094017094017%;*width:40.11774868157847%}.row-fluid .span4{width:31.623931623931625%;*width:31.570740134569924%}.row-fluid .span3{width:23.076923076923077%;*width:23.023731587561375%}.row-fluid .span2{width:14.52991452991453%;*width:14.476723040552828%}.row-fluid .span1{width:5.982905982905983%;*width:5.929714493544281%}.row-fluid .offset12{margin-left:105.12820512820512%;*margin-left:105.02182214948171%}.row-fluid .offset12:first-child{margin-left:102.56410256410257%;*margin-left:102.45771958537915%}.row-fluid .offset11{margin-left:96.58119658119658%;*margin-left:96.47481360247316%}.row-fluid .offset11:first-child{margin-left:94.01709401709402%;*margin-left:93.91071103837061%}.row-fluid .offset10{margin-left:88.03418803418803%;*margin-left:87.92780505546462%}.row-fluid .offset10:first-child{margin-left:85.47008547008548%;*margin-left:85.36370249136206%}.row-fluid .offset9{margin-left:79.48717948717949%;*margin-left:79.38079650845607%}.row-fluid .offset9:first-child{margin-left:76.92307692307693%;*margin-left:76.81669394435352%}.row-fluid .offset8{margin-left:70.94017094017094%;*margin-left:70.83378796144753%}.row-fluid .offset8:first-child{margin-left:68.37606837606839%;*margin-left:68.26968539734497%}.row-fluid .offset7{margin-left:62.393162393162385%;*margin-left:62.28677941443899%}.row-fluid .offset7:first-child{margin-left:59.82905982905982%;*margin-left:59.72267685033642%}.row-fluid .offset6{margin-left:53.84615384615384%;*margin-left:53.739770867430444%}.row-fluid .offset6:first-child{margin-left:51.28205128205128%;*margin-left:51.175668303327875%}.row-fluid .offset5{margin-left:45.299145299145295%;*margin-left:45.1927623204219%}.row-fluid .offset5:first-child{margin-left:42.73504273504273%;*margin-left:42.62865975631933%}.row-fluid .offset4{margin-left:36.75213675213675%;*margin-left:36.645753773413354%}.row-fluid .offset4:first-child{margin-left:34.18803418803419%;*margin-left:34.081651209310785%}.row-fluid .offset3{margin-left:28.205128205128204%;*margin-left:28.0987452264048%}.row-fluid .offset3:first-child{margin-left:25.641025641025642%;*margin-left:25.53464266230224%}.row-fluid .offset2{margin-left:19.65811965811966%;*margin-left:19.551736679396257%}.row-fluid .offset2:first-child{margin-left:17.094017094017094%;*margin-left:16.98763411529369%}.row-fluid .offset1{margin-left:11.11111111111111%;*margin-left:11.004728132387708%}.row-fluid .offset1:first-child{margin-left:8.547008547008547%;*margin-left:8.440625568285142%}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:30px}input.span12,textarea.span12,.uneditable-input.span12{width:1156px}input.span11,textarea.span11,.uneditable-input.span11{width:1056px}input.span10,textarea.span10,.uneditable-input.span10{width:956px}input.span9,textarea.span9,.uneditable-input.span9{width:856px}input.span8,textarea.span8,.uneditable-input.span8{width:756px}input.span7,textarea.span7,.uneditable-input.span7{width:656px}input.span6,textarea.span6,.uneditable-input.span6{width:556px}input.span5,textarea.span5,.uneditable-input.span5{width:456px}input.span4,textarea.span4,.uneditable-input.span4{width:356px}input.span3,textarea.span3,.uneditable-input.span3{width:256px}input.span2,textarea.span2,.uneditable-input.span2{width:156px}input.span1,textarea.span1,.uneditable-input.span1{width:56px}.thumbnails{margin-left:-30px}.thumbnails>li{margin-left:30px}.row-fluid .thumbnails{margin-left:0}}@media(min-width:768px) and (max-width:979px){.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:724px}.span12{width:724px}.span11{width:662px}.span10{width:600px}.span9{width:538px}.span8{width:476px}.span7{width:414px}.span6{width:352px}.span5{width:290px}.span4{width:228px}.span3{width:166px}.span2{width:104px}.span1{width:42px}.offset12{margin-left:764px}.offset11{margin-left:702px}.offset10{margin-left:640px}.offset9{margin-left:578px}.offset8{margin-left:516px}.offset7{margin-left:454px}.offset6{margin-left:392px}.offset5{margin-left:330px}.offset4{margin-left:268px}.offset3{margin-left:206px}.offset2{margin-left:144px}.offset1{margin-left:82px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.7624309392265194%;*margin-left:2.709239449864817%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.7624309392265194%}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.43646408839778%;*width:91.38327259903608%}.row-fluid .span10{width:82.87292817679558%;*width:82.81973668743387%}.row-fluid .span9{width:74.30939226519337%;*width:74.25620077583166%}.row-fluid .span8{width:65.74585635359117%;*width:65.69266486422946%}.row-fluid .span7{width:57.18232044198895%;*width:57.12912895262725%}.row-fluid .span6{width:48.61878453038674%;*width:48.56559304102504%}.row-fluid .span5{width:40.05524861878453%;*width:40.00205712942283%}.row-fluid .span4{width:31.491712707182323%;*width:31.43852121782062%}.row-fluid .span3{width:22.92817679558011%;*width:22.87498530621841%}.row-fluid .span2{width:14.3646408839779%;*width:14.311449394616199%}.row-fluid .span1{width:5.801104972375691%;*width:5.747913483013988%}.row-fluid .offset12{margin-left:105.52486187845304%;*margin-left:105.41847889972962%}.row-fluid .offset12:first-child{margin-left:102.76243093922652%;*margin-left:102.6560479605031%}.row-fluid .offset11{margin-left:96.96132596685082%;*margin-left:96.8549429881274%}.row-fluid .offset11:first-child{margin-left:94.1988950276243%;*margin-left:94.09251204890089%}.row-fluid .offset10{margin-left:88.39779005524862%;*margin-left:88.2914070765252%}.row-fluid .offset10:first-child{margin-left:85.6353591160221%;*margin-left:85.52897613729868%}.row-fluid .offset9{margin-left:79.8342541436464%;*margin-left:79.72787116492299%}.row-fluid .offset9:first-child{margin-left:77.07182320441989%;*margin-left:76.96544022569647%}.row-fluid .offset8{margin-left:71.2707182320442%;*margin-left:71.16433525332079%}.row-fluid .offset8:first-child{margin-left:68.50828729281768%;*margin-left:68.40190431409427%}.row-fluid .offset7{margin-left:62.70718232044199%;*margin-left:62.600799341718584%}.row-fluid .offset7:first-child{margin-left:59.94475138121547%;*margin-left:59.838368402492065%}.row-fluid .offset6{margin-left:54.14364640883978%;*margin-left:54.037263430116376%}.row-fluid .offset6:first-child{margin-left:51.38121546961326%;*margin-left:51.27483249088986%}.row-fluid .offset5{margin-left:45.58011049723757%;*margin-left:45.47372751851417%}.row-fluid .offset5:first-child{margin-left:42.81767955801105%;*margin-left:42.71129657928765%}.row-fluid .offset4{margin-left:37.01657458563536%;*margin-left:36.91019160691196%}.row-fluid .offset4:first-child{margin-left:34.25414364640884%;*margin-left:34.14776066768544%}.row-fluid .offset3{margin-left:28.45303867403315%;*margin-left:28.346655695309746%}.row-fluid .offset3:first-child{margin-left:25.69060773480663%;*margin-left:25.584224756083227%}.row-fluid .offset2{margin-left:19.88950276243094%;*margin-left:19.783119783707537%}.row-fluid .offset2:first-child{margin-left:17.12707182320442%;*margin-left:17.02068884448102%}.row-fluid .offset1{margin-left:11.32596685082873%;*margin-left:11.219583872105325%}.row-fluid .offset1:first-child{margin-left:8.56353591160221%;*margin-left:8.457152932878806%}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:710px}input.span11,textarea.span11,.uneditable-input.span11{width:648px}input.span10,textarea.span10,.uneditable-input.span10{width:586px}input.span9,textarea.span9,.uneditable-input.span9{width:524px}input.span8,textarea.span8,.uneditable-input.span8{width:462px}input.span7,textarea.span7,.uneditable-input.span7{width:400px}input.span6,textarea.span6,.uneditable-input.span6{width:338px}input.span5,textarea.span5,.uneditable-input.span5{width:276px}input.span4,textarea.span4,.uneditable-input.span4{width:214px}input.span3,textarea.span3,.uneditable-input.span3{width:152px}input.span2,textarea.span2,.uneditable-input.span2{width:90px}input.span1,textarea.span1,.uneditable-input.span1{width:28px}}@media(max-width:767px){body{padding-right:20px;padding-left:20px}.navbar-fixed-top,.navbar-fixed-bottom,.navbar-static-top{margin-right:-20px;margin-left:-20px}.container-fluid{padding:0}.dl-horizontal dt{float:none;width:auto;clear:none;text-align:left}.dl-horizontal dd{margin-left:0}.container{width:auto}.row-fluid{width:100%}.row,.thumbnails{margin-left:0}.thumbnails>li{float:none;margin-left:0}[class*="span"],.uneditable-input[class*="span"],.row-fluid [class*="span"]{display:block;float:none;width:100%;margin-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.span12,.row-fluid .span12{width:100%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="offset"]:first-child{margin-left:0}.input-large,.input-xlarge,.input-xxlarge,input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.input-prepend input,.input-append input,.input-prepend input[class*="span"],.input-append input[class*="span"]{display:inline-block;width:auto}.controls-row [class*="span"]+[class*="span"]{margin-left:0}.modal{position:fixed;top:20px;right:20px;left:20px;width:auto;margin:0}.modal.fade{top:-100px}.modal.fade.in{top:20px}}@media(max-width:480px){.nav-collapse{-webkit-transform:translate3d(0,0,0)}.page-header h1 small{display:block;line-height:20px}input[type="checkbox"],input[type="radio"]{border:1px solid #ccc}.form-horizontal .control-label{float:none;width:auto;padding-top:0;text-align:left}.form-horizontal .controls{margin-left:0}.form-horizontal .control-list{padding-top:0}.form-horizontal .form-actions{padding-right:10px;padding-left:10px}.media .pull-left,.media .pull-right{display:block;float:none;margin-bottom:10px}.media-object{margin-right:0;margin-left:0}.modal{top:10px;right:10px;left:10px}.modal-header .close{padding:10px;margin:-10px}.carousel-caption{position:static}}@media(max-width:979px){body{padding-top:0}.navbar-fixed-top,.navbar-fixed-bottom{position:static}.navbar-fixed-top{margin-bottom:20px}.navbar-fixed-bottom{margin-top:20px}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding:5px}.navbar .container{width:auto;padding:0}.navbar .brand{padding-right:10px;padding-left:10px;margin:0 0 0 -5px}.nav-collapse{clear:both}.nav-collapse .nav{float:none;margin:0 0 10px}.nav-collapse .nav>li{float:none}.nav-collapse .nav>li>a{margin-bottom:2px}.nav-collapse .nav>.divider-vertical{display:none}.nav-collapse .nav .nav-header{color:#777;text-shadow:none}.nav-collapse .nav>li>a,.nav-collapse .dropdown-menu a{padding:9px 15px;font-weight:bold;color:#777;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.nav-collapse .btn{padding:4px 10px 4px;font-weight:normal;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.nav-collapse .dropdown-menu li+li a{margin-bottom:2px}.nav-collapse .nav>li>a:hover,.nav-collapse .nav>li>a:focus,.nav-collapse .dropdown-menu a:hover,.nav-collapse .dropdown-menu a:focus{background-color:#f2f2f2}.navbar-inverse .nav-collapse .nav>li>a,.navbar-inverse .nav-collapse .dropdown-menu a{color:#999}.navbar-inverse .nav-collapse .nav>li>a:hover,.navbar-inverse .nav-collapse .nav>li>a:focus,.navbar-inverse .nav-collapse .dropdown-menu a:hover,.navbar-inverse .nav-collapse .dropdown-menu a:focus{background-color:#111}.nav-collapse.in .btn-group{padding:0;margin-top:5px}.nav-collapse .dropdown-menu{position:static;top:auto;left:auto;display:none;float:none;max-width:none;padding:0;margin:0 15px;background-color:transparent;border:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.nav-collapse .open>.dropdown-menu{display:block}.nav-collapse .dropdown-menu:before,.nav-collapse .dropdown-menu:after{display:none}.nav-collapse .dropdown-menu .divider{display:none}.nav-collapse .nav>li>.dropdown-menu:before,.nav-collapse .nav>li>.dropdown-menu:after{display:none}.nav-collapse .navbar-form,.nav-collapse .navbar-search{float:none;padding:10px 15px;margin:10px 0;border-top:1px solid #f2f2f2;border-bottom:1px solid #f2f2f2;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1)}.navbar-inverse .nav-collapse .navbar-form,.navbar-inverse .nav-collapse .navbar-search{border-top-color:#111;border-bottom-color:#111}.navbar .nav-collapse .nav.pull-right{float:none;margin-left:0}.nav-collapse,.nav-collapse.collapse{height:0;overflow:hidden}.navbar .btn-navbar{display:block}.navbar-static .navbar-inner{padding-right:10px;padding-left:10px}}@media(min-width:980px){.nav-collapse.collapse{height:auto!important;overflow:visible!important}} diff --git a/core/src/main/resources/spark/ui/static/bootstrap.min.css b/core/src/main/resources/spark/ui/static/bootstrap.min.css index b74b4546a6..b6428e6958 100644 --- a/core/src/main/resources/spark/ui/static/bootstrap.min.css +++ b/core/src/main/resources/spark/ui/static/bootstrap.min.css @@ -1,9 +1,9 @@ /*! - * Bootstrap v2.0.4 + * Bootstrap v2.3.2 * * Copyright 2012 Twitter, Inc * Licensed under the Apache License v2.0 * http://www.apache.org/licenses/LICENSE-2.0 * * Designed and built with all the love in the world @twitter by @mdo and @fat. - */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:28px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;line-height:18px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:28px;margin-left:2.127659574%;*margin-left:2.0744680846382977%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:99.99999998999999%;*width:99.94680850063828%}.row-fluid .span11{width:91.489361693%;*width:91.4361702036383%}.row-fluid .span10{width:82.97872339599999%;*width:82.92553190663828%}.row-fluid .span9{width:74.468085099%;*width:74.4148936096383%}.row-fluid .span8{width:65.95744680199999%;*width:65.90425531263828%}.row-fluid .span7{width:57.446808505%;*width:57.3936170156383%}.row-fluid .span6{width:48.93617020799999%;*width:48.88297871863829%}.row-fluid .span5{width:40.425531911%;*width:40.3723404216383%}.row-fluid .span4{width:31.914893614%;*width:31.8617021246383%}.row-fluid .span3{width:23.404255317%;*width:23.3510638276383%}.row-fluid .span2{width:14.89361702%;*width:14.8404255306383%}.row-fluid .span1{width:6.382978723%;*width:6.329787233638298%}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;content:""}.container-fluid:after{clear:both}p{margin:0 0 9px}p small{font-size:11px;color:#999}.lead{margin-bottom:18px;font-size:20px;font-weight:200;line-height:27px}h1,h2,h3,h4,h5,h6{margin:0;font-family:inherit;font-weight:bold;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;color:#999}h1{font-size:30px;line-height:36px}h1 small{font-size:18px}h2{font-size:24px;line-height:36px}h2 small{font-size:18px}h3{font-size:18px;line-height:27px}h3 small{font-size:14px}h4,h5,h6{line-height:18px}h4{font-size:14px}h4 small{font-size:12px}h5{font-size:12px}h6{font-size:11px;color:#999;text-transform:uppercase}.page-header{padding-bottom:17px;margin:18px 0;border-bottom:1px solid #eee}.page-header h1{line-height:1}ul,ol{padding:0;margin:0 0 9px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}ul{list-style:disc}ol{list-style:decimal}li{line-height:18px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:18px}dt,dd{line-height:18px}dt{font-weight:bold;line-height:17px}dd{margin-left:9px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:18px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}strong{font-weight:bold}em{font-style:italic}.muted{color:#999}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 18px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:22.5px}blockquote small{display:block;line-height:18px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:18px;font-style:normal;line-height:18px}small{font-size:100%}cite{font-style:normal}code,pre{padding:0 3px 2px;font-family:Menlo,Monaco,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:8.5px;margin:0 0 9px;font-size:12.025px;line-height:18px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:18px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 18px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:27px;font-size:19.5px;line-height:36px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:13.5px;color:#999}label,input,button,select,textarea{font-size:13px;font-weight:normal;line-height:18px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:18px;padding:4px;margin-bottom:9px;font-size:13px;line-height:18px;color:#555}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-ms-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:3px 0;*margin-top:0;line-height:normal;cursor:pointer}input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}.uneditable-textarea{width:auto;height:auto}select,input[type="file"]{height:28px;*margin-top:4px;line-height:28px}select{width:220px;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}input.span12,textarea.span12,.uneditable-input.span12{width:930px}input.span11,textarea.span11,.uneditable-input.span11{width:850px}input.span10,textarea.span10,.uneditable-input.span10{width:770px}input.span9,textarea.span9,.uneditable-input.span9{width:690px}input.span8,textarea.span8,.uneditable-input.span8{width:610px}input.span7,textarea.span7,.uneditable-input.span7{width:530px}input.span6,textarea.span6,.uneditable-input.span6{width:450px}input.span5,textarea.span5,.uneditable-input.span5{width:370px}input.span4,textarea.span4,.uneditable-input.span4{width:290px}input.span3,textarea.span3,.uneditable-input.span3{width:210px}input.span2,textarea.span2,.uneditable-input.span2{width:130px}input.span1,textarea.span1,.uneditable-input.span1{width:50px}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee;border-color:#ddd}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:0 0 6px #dbc59e;-moz-box-shadow:0 0 6px #dbc59e;box-shadow:0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:0 0 6px #d59392;-moz-box-shadow:0 0 6px #d59392;box-shadow:0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:0 0 6px #7aba7b;-moz-box-shadow:0 0 6px #7aba7b;box-shadow:0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:17px 20px 18px;margin-top:18px;margin-bottom:18px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;content:""}.form-actions:after{clear:both}.uneditable-input{overflow:hidden;white-space:nowrap;cursor:not-allowed;background-color:#fff;border-color:#eee;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}:-moz-placeholder{color:#999}:-ms-input-placeholder{color:#999}::-webkit-input-placeholder{color:#999}.help-block,.help-inline{color:#555}.help-block{display:block;margin-bottom:9px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-prepend,.input-append{margin-bottom:5px}.input-prepend input,.input-append input,.input-prepend select,.input-append select,.input-prepend .uneditable-input,.input-append .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;vertical-align:middle;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend input:focus,.input-append input:focus,.input-prepend select:focus,.input-append select:focus,.input-prepend .uneditable-input:focus,.input-append .uneditable-input:focus{z-index:2}.input-prepend .uneditable-input,.input-append .uneditable-input{border-left-color:#ccc}.input-prepend .add-on,.input-append .add-on{display:inline-block;width:auto;height:18px;min-width:16px;padding:4px 5px;font-weight:normal;line-height:18px;text-align:center;text-shadow:0 1px 0 #fff;vertical-align:middle;background-color:#eee;border:1px solid #ccc}.input-prepend .add-on,.input-append .add-on,.input-prepend .btn,.input-append .btn{margin-left:-1px;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend .active,.input-append .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .uneditable-input{border-right-color:#ccc;border-left-color:#eee}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:14px;-moz-border-radius:14px;border-radius:14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:9px}legend+.control-group{margin-top:18px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:18px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:9px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:18px}.table th,.table td{padding:8px;line-height:18px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapsed;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table tbody tr:hover td,.table tbody tr:hover th{background-color:#f5f5f5}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}[class^="icon-"]:last-child,[class*=" icon-"]:last-child{*margin-left:0}.icon-white{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:"";opacity:.3;filter:alpha(opacity=30)}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown:hover .caret,.open .caret{opacity:1;filter:alpha(opacity=100)}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:4px 0;margin:1px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:8px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 15px;clear:both;font-weight:normal;line-height:18px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#08c}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #eee;border:1px solid rgba(0,0,0,0.05);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-ms-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-ms-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:18px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 10px 4px;margin-bottom:0;*margin-left:.3em;font-size:13px;line-height:18px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-ms-linear-gradient(top,#fff,#e6e6e6);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(top,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #ccc;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#b3b3b3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffff',endColorstr='#e6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-ms-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:15px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:1px}.btn-small{padding:5px 9px;font-size:11px;line-height:16px}.btn-small [class^="icon-"]{margin-top:-1px}.btn-mini{padding:2px 6px;font-size:11px;line-height:14px}.btn-primary,.btn-primary:hover,.btn-warning,.btn-warning:hover,.btn-danger,.btn-danger:hover,.btn-success,.btn-success:hover,.btn-info,.btn-info:hover,.btn-inverse,.btn-inverse:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#ccc;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25)}.btn-primary{background-color:#0074cc;*background-color:#05c;background-image:-ms-linear-gradient(top,#08c,#05c);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#05c));background-image:-webkit-linear-gradient(top,#08c,#05c);background-image:-o-linear-gradient(top,#08c,#05c);background-image:-moz-linear-gradient(top,#08c,#05c);background-image:linear-gradient(top,#08c,#05c);background-repeat:repeat-x;border-color:#05c #05c #003580;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#0088cc',endColorstr='#0055cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{background-color:#05c;*background-color:#004ab3}.btn-primary:active,.btn-primary.active{background-color:#004099 \9}.btn-warning{background-color:#faa732;*background-color:#f89406;background-image:-ms-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fbb450',endColorstr='#f89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{background-color:#da4f49;*background-color:#bd362f;background-image:-ms-linear-gradient(top,#ee5f5b,#bd362f);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ee5f5b',endColorstr='#bd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{background-color:#5bb75b;*background-color:#51a351;background-image:-ms-linear-gradient(top,#62c462,#51a351);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#62c462',endColorstr='#51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{background-color:#49afcd;*background-color:#2f96b4;background-image:-ms-linear-gradient(top,#5bc0de,#2f96b4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#5bc0de',endColorstr='#2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{background-color:#414141;*background-color:#222;background-image:-ms-linear-gradient(top,#555,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#555),to(#222));background-image:-webkit-linear-gradient(top,#555,#222);background-image:-o-linear-gradient(top,#555,#222);background-image:-moz-linear-gradient(top,#555,#222);background-image:linear-gradient(top,#555,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#555555',endColorstr='#222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:2px;*padding-bottom:2px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-group{position:relative;*margin-left:.3em;*zoom:1}.btn-group:before,.btn-group:after{display:table;content:""}.btn-group:after{clear:both}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:9px;margin-bottom:9px}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-group>.btn{position:relative;float:left;margin-left:-1px;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.dropdown-toggle{*padding-top:4px;padding-right:8px;*padding-bottom:4px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini.dropdown-toggle{padding-right:5px;padding-left:5px}.btn-group>.btn-small.dropdown-toggle{*padding-top:4px;*padding-bottom:4px}.btn-group>.btn-large.dropdown-toggle{padding-right:12px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#05c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:7px;margin-left:0}.btn:hover .caret,.open.btn-group .caret{opacity:1;filter:alpha(opacity=100)}.btn-mini .caret{margin-top:5px}.btn-small .caret{margin-top:6px}.btn-large .caret{margin-top:6px;border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:.75;filter:alpha(opacity=75)}.alert{padding:8px 35px 8px 14px;margin-bottom:18px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert-heading{color:inherit}.alert .close{position:relative;top:-2px;right:-21px;line-height:18px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:18px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav .nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:18px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:8px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:18px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 5px 5px;-moz-border-radius:0 0 5px 5px;border-radius:0 0 5px 5px}.nav-pills .dropdown-menu{-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.nav-tabs .dropdown-toggle .caret,.nav-pills .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav-tabs .dropdown-toggle:hover .caret,.nav-pills .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .active .dropdown-toggle .caret,.nav-pills .active .dropdown-toggle .caret{border-top-color:#333;border-bottom-color:#333}.nav>.dropdown.active>a:hover{color:#000;cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.navbar{*position:relative;*z-index:2;margin-bottom:18px;overflow:visible}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#2c2c2c;background-image:-moz-linear-gradient(top,#333,#222);background-image:-ms-linear-gradient(top,#333,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#333),to(#222));background-image:-webkit-linear-gradient(top,#333,#222);background-image:-o-linear-gradient(top,#333,#222);background-image:linear-gradient(top,#333,#222);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#333333',endColorstr='#222222',GradientType=0);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.25),inset 0 -1px 0 rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.25),inset 0 -1px 0 rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.25),inset 0 -1px 0 rgba(0,0,0,0.1)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar{color:#999}.navbar .brand:hover{text-decoration:none}.navbar .brand{display:block;float:left;padding:8px 20px 12px;margin-left:-20px;font-size:20px;font-weight:200;line-height:1;color:#999}.navbar .navbar-text{margin-bottom:0;line-height:40px}.navbar .navbar-link{color:#999}.navbar .navbar-link:hover{color:#fff}.navbar .btn,.navbar .btn-group{margin-top:5px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:6px;margin-bottom:0}.navbar-search .search-query{padding:4px 9px;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;color:#fff;background-color:#626262;border:1px solid #151515;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-ms-transition:none;-o-transition:none;transition:none}.navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-search .search-query:focus,.navbar-search .search-query.focused{padding:5px 10px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-bottom{bottom:0}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{display:block;float:left}.navbar .nav>li>a{float:none;padding:9px 10px 11px;line-height:19px;color:#999;text-decoration:none;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar .btn{display:inline-block;padding:4px 10px 4px;margin:5px 5px 6px;line-height:18px}.navbar .btn-group{padding:5px 5px 6px;margin:0}.navbar .nav>li>a:hover{color:#fff;text-decoration:none;background-color:transparent}.navbar .nav .active>a,.navbar .nav .active>a:hover{color:#fff;text-decoration:none;background-color:#222}.navbar .divider-vertical{width:1px;height:40px;margin:0 9px;overflow:hidden;background-color:#222;border-right:1px solid #333}.navbar .nav.pull-right{margin-right:0;margin-left:10px}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;background-color:#2c2c2c;*background-color:#222;background-image:-ms-linear-gradient(top,#333,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#333),to(#222));background-image:-webkit-linear-gradient(top,#333,#222);background-image:-o-linear-gradient(top,#333,#222);background-image:linear-gradient(top,#333,#222);background-image:-moz-linear-gradient(top,#333,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#333333',endColorstr='#222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{background-color:#222;*background-color:#151515}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#080808 \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown .dropdown-toggle .caret,.navbar .nav li.dropdown.open .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar .nav li.dropdown.active .caret{opacity:1;filter:alpha(opacity=100)}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{background-color:transparent}.navbar .nav li.dropdown.active>.dropdown-toggle:hover{color:#fff}.navbar .pull-right .dropdown-menu,.navbar .dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right .dropdown-menu:before,.navbar .dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right .dropdown-menu:after,.navbar .dropdown-menu.pull-right:after{right:13px;left:auto}.breadcrumb{padding:7px 14px;margin:0 0 18px;list-style:none;background-color:#fbfbfb;background-image:-moz-linear-gradient(top,#fff,#f5f5f5);background-image:-ms-linear-gradient(top,#fff,#f5f5f5);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f5f5f5));background-image:-webkit-linear-gradient(top,#fff,#f5f5f5);background-image:-o-linear-gradient(top,#fff,#f5f5f5);background-image:linear-gradient(top,#fff,#f5f5f5);background-repeat:repeat-x;border:1px solid #ddd;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffff',endColorstr='#f5f5f5',GradientType=0);-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#999}.breadcrumb .active a{color:#333}.pagination{height:36px;margin:18px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a{float:left;padding:0 14px;line-height:34px;text-decoration:none;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a{background-color:#f5f5f5}.pagination .active a{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin-bottom:18px;margin-left:0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2070}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-ms-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1020;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-2px}.tooltip.right{margin-left:2px}.tooltip.bottom{margin-top:2px}.tooltip.left{margin-left:-2px}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top:5px solid #000;border-right:5px solid transparent;border-left:5px solid transparent}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-top:5px solid transparent;border-bottom:5px solid transparent;border-left:5px solid #000}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-right:5px solid transparent;border-bottom:5px solid #000;border-left:5px solid transparent}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-top:5px solid transparent;border-right:5px solid #000;border-bottom:5px solid transparent}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;padding:5px}.popover.top{margin-top:-5px}.popover.right{margin-left:5px}.popover.bottom{margin-top:5px}.popover.left{margin-left:-5px}.popover.top .arrow{bottom:0;left:50%;margin-left:-5px;border-top:5px solid #000;border-right:5px solid transparent;border-left:5px solid transparent}.popover.right .arrow{top:50%;left:0;margin-top:-5px;border-top:5px solid transparent;border-right:5px solid #000;border-bottom:5px solid transparent}.popover.bottom .arrow{top:0;left:50%;margin-left:-5px;border-right:5px solid transparent;border-bottom:5px solid #000;border-left:5px solid transparent}.popover.left .arrow{top:50%;right:0;margin-top:-5px;border-top:5px solid transparent;border-bottom:5px solid transparent;border-left:5px solid #000}.popover .arrow{position:absolute;width:0;height:0}.popover-inner{width:280px;padding:3px;overflow:hidden;background:#000;background:rgba(0,0,0,0.8);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3)}.popover-title{padding:9px 15px;line-height:1;background-color:#f5f5f5;border-bottom:1px solid #eee;-webkit-border-radius:3px 3px 0 0;-moz-border-radius:3px 3px 0 0;border-radius:3px 3px 0 0}.popover-content{padding:14px;background-color:#fff;-webkit-border-radius:0 0 3px 3px;-moz-border-radius:0 0 3px 3px;border-radius:0 0 3px 3px;-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:18px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:1;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:0 1px 1px rgba(0,0,0,0.075);box-shadow:0 1px 1px rgba(0,0,0,0.075)}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px}.label,.badge{font-size:10.998px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:18px;margin-bottom:18px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-ms-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(top,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#f5f5f5',endColorstr='#f9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{width:0;height:18px;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(top,#149bdf,#0480be);background-image:-ms-linear-gradient(top,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#149bdf',endColorstr='#0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-ms-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress-striped .bar{background-color:#149bdf;background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-ms-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(top,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ee5f5b',endColorstr='#c43c35',GradientType=0)}.progress-danger.progress-striped .bar{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-ms-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(top,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#62c462',endColorstr='#57a957',GradientType=0)}.progress-success.progress-striped .bar{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-ms-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(top,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#5bc0de',endColorstr='#339bb9',GradientType=0)}.progress-info.progress-striped .bar{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-ms-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fbb450',endColorstr='#f89406',GradientType=0)}.progress-warning.progress-striped .bar{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:18px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:18px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-ms-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:10px 15px 5px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{color:#fff}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:27px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden} + */.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{width:auto\9;height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img,.google-maps img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,html input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}label,select,button,input[type="button"],input[type="reset"],input[type="submit"],input[type="radio"],input[type="checkbox"]{cursor:pointer}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}@media print{*{color:#000!important;text-shadow:none!important;background:transparent!important;box-shadow:none!important}a,a:visited{text-decoration:underline}a[href]:after{content:" (" attr(href) ")"}abbr[title]:after{content:" (" attr(title) ")"}.ir a:after,a[href^="javascript:"]:after,a[href^="#"]:after{content:""}pre,blockquote{border:1px solid #999;page-break-inside:avoid}thead{display:table-header-group}tr,img{page-break-inside:avoid}img{max-width:100%!important}@page{margin:.5cm}p,h2,h3{orphans:3;widows:3}h2,h3{page-break-after:avoid}}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover,a:focus{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.127659574468085%}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:21px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}a.muted:hover,a.muted:focus{color:#808080}.text-warning{color:#c09853}a.text-warning:hover,a.text-warning:focus{color:#a47e3c}.text-error{color:#b94a48}a.text-error:hover,a.text-error:focus{color:#953b39}.text-info{color:#3a87ad}a.text-info:hover,a.text-info:focus{color:#2d6987}.text-success{color:#468847}a.text-success:hover,a.text-success:focus{color:#356635}.text-left{text-align:left}.text-right{text-align:right}.text-center{text-align:center}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:20px;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1,h2,h3{line-height:40px}h1{font-size:38.5px}h2{font-size:31.5px}h3{font-size:24.5px}h4{font-size:17.5px}h5{font-size:14px}h6{font-size:11.9px}h1 small{font-size:24.5px}h2 small{font-size:17.5px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}ul.inline,ol.inline{margin-left:0;list-style:none}ul.inline>li,ol.inline>li{display:inline-block;*display:inline;padding-right:5px;padding-left:5px;*zoom:1}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal{*zoom:1}.dl-horizontal:before,.dl-horizontal:after{display:table;line-height:0;content:""}.dl-horizontal:after{clear:both}.dl-horizontal dt{float:left;width:160px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:180px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title],abbr[data-original-title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:17.5px;font-weight:300;line-height:1.25}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;white-space:nowrap;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;white-space:pre;white-space:pre-wrap;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:10px;font-size:14px;line-height:20px;color:#555;vertical-align:middle;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}input,textarea,.uneditable-input{width:206px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #ccc}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:20px;padding-left:20px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-20px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"],.row-fluid .controls-row [class*="span"]{float:left}.controls-row .checkbox[class*="span"],.controls-row .radio[class*="span"]{padding-top:5px}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning .control-label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853}.control-group.warning input,.control-group.warning select,.control-group.warning textarea{border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error .control-label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48}.control-group.error input,.control-group.error select,.control-group.error textarea{border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success .control-label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847}.control-group.success input,.control-group.success select,.control-group.success textarea{border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}.control-group.info .control-label,.control-group.info .help-block,.control-group.info .help-inline{color:#3a87ad}.control-group.info .checkbox,.control-group.info .radio,.control-group.info input,.control-group.info select,.control-group.info textarea{color:#3a87ad}.control-group.info input,.control-group.info select,.control-group.info textarea{border-color:#3a87ad;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.info input:focus,.control-group.info select:focus,.control-group.info textarea:focus{border-color:#2d6987;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3}.control-group.info .input-prepend .add-on,.control-group.info .input-append .add-on{color:#3a87ad;background-color:#d9edf7;border-color:#3a87ad}input:focus:invalid,textarea:focus:invalid,select:focus:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:invalid:focus,textarea:focus:invalid:focus,select:focus:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{display:inline-block;margin-bottom:10px;font-size:0;white-space:nowrap;vertical-align:middle}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input,.input-append .dropdown-menu,.input-prepend .dropdown-menu,.input-append .popover,.input-prepend .popover{font-size:14px}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;vertical-align:top;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn,.input-append .btn-group>.dropdown-toggle,.input-prepend .btn-group>.dropdown-toggle{vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-append input+.btn-group .btn:last-child,.input-append select+.btn-group .btn:last-child,.input-append .uneditable-input+.btn-group .btn:last-child{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-append .add-on,.input-append .btn,.input-append .btn-group{margin-left:-1px}.input-append .add-on:last-child,.input-append .btn:last-child,.input-append .btn-group:last-child>.dropdown-toggle{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append input+.btn-group .btn,.input-prepend.input-append select+.btn-group .btn,.input-prepend.input-append .uneditable-input+.btn-group .btn{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append .btn-group:first-child{margin-left:0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:160px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:180px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:180px}.form-horizontal .help-block{margin-bottom:0}.form-horizontal input+.help-block,.form-horizontal select+.help-block,.form-horizontal textarea+.help-block,.form-horizontal .uneditable-input+.help-block,.form-horizontal .input-prepend+.help-block,.form-horizontal .input-append+.help-block{margin-top:10px}.form-horizontal .form-actions{padding-left:180px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table .table{background-color:#fff}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child>th:first-child,.table-bordered tbody:first-child tr:first-child>td:first-child,.table-bordered tbody:first-child tr:first-child>th:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child>th:last-child,.table-bordered tbody:first-child tr:first-child>td:last-child,.table-bordered tbody:first-child tr:first-child>th:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child>th:first-child,.table-bordered tbody:last-child tr:last-child>td:first-child,.table-bordered tbody:last-child tr:last-child>th:first-child,.table-bordered tfoot:last-child tr:last-child>td:first-child,.table-bordered tfoot:last-child tr:last-child>th:first-child{-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child>th:last-child,.table-bordered tbody:last-child tr:last-child>td:last-child,.table-bordered tbody:last-child tr:last-child>th:last-child,.table-bordered tfoot:last-child tr:last-child>td:last-child,.table-bordered tfoot:last-child tr:last-child>th:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered tfoot+tbody:last-child tr:last-child td:first-child{-webkit-border-bottom-left-radius:0;border-bottom-left-radius:0;-moz-border-radius-bottomleft:0}.table-bordered tfoot+tbody:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:0;border-bottom-right-radius:0;-moz-border-radius-bottomright:0}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-striped tbody>tr:nth-child(odd)>td,.table-striped tbody>tr:nth-child(odd)>th{background-color:#f9f9f9}.table-hover tbody tr:hover>td,.table-hover tbody tr:hover>th{background-color:#f5f5f5}table td[class*="span"],table th[class*="span"],.row-fluid table td[class*="span"],.row-fluid table th[class*="span"]{display:table-cell;float:none;margin-left:0}.table td.span1,.table th.span1{float:none;width:44px;margin-left:0}.table td.span2,.table th.span2{float:none;width:124px;margin-left:0}.table td.span3,.table th.span3{float:none;width:204px;margin-left:0}.table td.span4,.table th.span4{float:none;width:284px;margin-left:0}.table td.span5,.table th.span5{float:none;width:364px;margin-left:0}.table td.span6,.table th.span6{float:none;width:444px;margin-left:0}.table td.span7,.table th.span7{float:none;width:524px;margin-left:0}.table td.span8,.table th.span8{float:none;width:604px;margin-left:0}.table td.span9,.table th.span9{float:none;width:684px;margin-left:0}.table td.span10,.table th.span10{float:none;width:764px;margin-left:0}.table td.span11,.table th.span11{float:none;width:844px;margin-left:0}.table td.span12,.table th.span12{float:none;width:924px;margin-left:0}.table tbody tr.success>td{background-color:#dff0d8}.table tbody tr.error>td{background-color:#f2dede}.table tbody tr.warning>td{background-color:#fcf8e3}.table tbody tr.info>td{background-color:#d9edf7}.table-hover tbody tr.success:hover>td{background-color:#d0e9c6}.table-hover tbody tr.error:hover>td{background-color:#ebcccc}.table-hover tbody tr.warning:hover>td{background-color:#faf2cc}.table-hover tbody tr.info:hover>td{background-color:#c4e3f3}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav-pills>.active>a>[class^="icon-"],.nav-pills>.active>a>[class*=" icon-"],.nav-list>.active>a>[class^="icon-"],.nav-list>.active>a>[class*=" icon-"],.navbar-inverse .nav>.active>a>[class^="icon-"],.navbar-inverse .nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:focus>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>li>a:focus>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"],.dropdown-submenu:hover>a>[class^="icon-"],.dropdown-submenu:focus>a>[class^="icon-"],.dropdown-submenu:hover>a>[class*=" icon-"],.dropdown-submenu:focus>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{width:16px;background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus,.dropdown-submenu:hover>a,.dropdown-submenu:focus>a{color:#fff;text-decoration:none;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#fff;text-decoration:none;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;outline:0;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu>.disabled>a,.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{color:#999}.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{text-decoration:none;cursor:default;background-color:transparent;background-image:none;filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.open{*z-index:1000}.open>.dropdown-menu{display:block}.dropdown-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:990}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:""}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover>.dropdown-menu{display:block}.dropup .dropdown-submenu>.dropdown-menu{top:auto;bottom:0;margin-top:0;margin-bottom:-2px;-webkit-border-radius:5px 5px 5px 0;-moz-border-radius:5px 5px 5px 0;border-radius:5px 5px 5px 0}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown-submenu.pull-left{float:none}.dropdown-submenu.pull-left>.dropdown-menu{left:-100%;margin-left:10px;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{z-index:1051;margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover,.close:focus{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 12px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #ccc;*border:0;border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-bottom-color:#b3b3b3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:focus,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover,.btn:focus{color:#333;text-decoration:none;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:11px 19px;font-size:17.5px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.btn-large [class^="icon-"],.btn-large [class*=" icon-"]{margin-top:4px}.btn-small{padding:2px 10px;font-size:11.9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.btn-small [class^="icon-"],.btn-small [class*=" icon-"]{margin-top:0}.btn-mini [class^="icon-"],.btn-mini [class*=" icon-"]{margin-top:-1px}.btn-mini{padding:0 6px;font-size:10.5px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}input[type="submit"].btn-block,input[type="reset"].btn-block,input[type="button"].btn-block{width:100%}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-moz-linear-gradient(top,#08c,#04c);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:focus,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:focus,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:focus,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-moz-linear-gradient(top,#62c462,#51a351);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:focus,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:focus,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-moz-linear-gradient(top,#444,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:focus,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active,.btn-link[disabled]{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover,.btn-link:focus{color:#005580;text-decoration:underline;background-color:transparent}.btn-link[disabled]:hover,.btn-link[disabled]:focus{color:#333;text-decoration:none}.btn-group{position:relative;display:inline-block;*display:inline;*margin-left:.3em;font-size:0;white-space:nowrap;vertical-align:middle;*zoom:1}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar>.btn+.btn,.btn-toolbar>.btn-group+.btn,.btn-toolbar>.btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu,.btn-group>.popover{font-size:14px}.btn-group>.btn-mini{font-size:10.5px}.btn-group>.btn-small{font-size:11.9px}.btn-group>.btn-large{font-size:17.5px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.btn-mini .caret,.btn-small .caret{margin-top:8px}.dropup .btn-large .caret{border-bottom-width:5px}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical>.btn{display:block;float:none;max-width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical>.btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical>.btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical>.btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical>.btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical>.btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert,.alert h4{color:#c09853}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-success h4{color:#468847}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-danger h4,.alert-error h4{color:#b94a48}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-info h4{color:#3a87ad}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover,.nav>li>a:focus{text-decoration:none;background-color:#eee}.nav>li>a>img{max-width:none}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover,.nav-list>.active>a:focus{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"],.nav-list [class*=" icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover,.nav-tabs>li>a:focus{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover,.nav-tabs>.active>a:focus{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover,.nav-pills>.active>a:focus{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover,.nav-tabs.nav-stacked>li>a:focus{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret,.nav .dropdown-toggle:focus .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover,.nav>.dropdown.active>a:focus{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover,.nav>li.dropdown.open.active>a:focus{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret,.nav li.dropdown.open a:focus .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover,.tabs-stacked .open>a:focus{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover,.tabs-below>.nav-tabs>li>a:focus{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover,.tabs-below>.nav-tabs>.active>a:focus{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover,.tabs-left>.nav-tabs>li>a:focus{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover,.tabs-left>.nav-tabs .active>a:focus{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover,.tabs-right>.nav-tabs>li>a:focus{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover,.tabs-right>.nav-tabs .active>a:focus{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover,.nav>.disabled>a:focus{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#f2f2f2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f2f2f2));background-image:-webkit-linear-gradient(top,#fff,#f2f2f2);background-image:-o-linear-gradient(top,#fff,#f2f2f2);background-image:linear-gradient(to bottom,#fff,#f2f2f2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fff2f2f2',GradientType=0);*zoom:1;-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar-inner:before,.navbar-inner:after{display:table;line-height:0;content:""}.navbar-inner:after{clear:both}.navbar .container{width:auto}.nav-collapse.collapse{height:auto;overflow:visible}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#777;text-shadow:0 1px 0 #fff}.navbar .brand:hover,.navbar .brand:focus{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px;color:#777}.navbar-link{color:#777}.navbar-link:hover,.navbar-link:focus{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:5px}.navbar .btn-group .btn,.navbar .input-prepend .btn,.navbar .input-append .btn,.navbar .input-prepend .btn-group,.navbar .input-append .btn-group{margin-top:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:5px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{border-width:0 0 1px}.navbar-fixed-bottom .navbar-inner{border-width:1px 0 0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 10px rgba(0,0,0,0.1);box-shadow:0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:0 -1px 10px rgba(0,0,0,0.1);box-shadow:0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right;margin-right:0}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#777;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:focus,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown>a:hover .caret,.navbar .nav li.dropdown>a:focus .caret{border-top-color:#333;border-bottom-color:#333}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#777;border-bottom-color:#777}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover,.navbar-inverse .brand:focus,.navbar-inverse .nav>li>a:focus{color:#fff}.navbar-inverse .brand{color:#999}.navbar-inverse .navbar-text{color:#999}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover,.navbar-inverse .navbar-link:focus{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>a:hover .caret,.navbar-inverse .nav li.dropdown>a:focus .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-moz-linear-gradient(top,#151515,#040404);background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:focus,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb>li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb>li>.divider{padding:0 5px;color:#ccc}.breadcrumb>.active{color:#999}.pagination{margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination ul>li{display:inline}.pagination ul>li>a,.pagination ul>li>span{float:left;padding:4px 12px;line-height:20px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination ul>li>a:hover,.pagination ul>li>a:focus,.pagination ul>.active>a,.pagination ul>.active>span{background-color:#f5f5f5}.pagination ul>.active>a,.pagination ul>.active>span{color:#999;cursor:default}.pagination ul>.disabled>span,.pagination ul>.disabled>a,.pagination ul>.disabled>a:hover,.pagination ul>.disabled>a:focus{color:#999;cursor:default;background-color:transparent}.pagination ul>li:first-child>a,.pagination ul>li:first-child>span{border-left-width:1px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.pagination ul>li:last-child>a,.pagination ul>li:last-child>span{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pagination-large ul>li>a,.pagination-large ul>li>span{padding:11px 19px;font-size:17.5px}.pagination-large ul>li:first-child>a,.pagination-large ul>li:first-child>span{-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.pagination-large ul>li:last-child>a,.pagination-large ul>li:last-child>span{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.pagination-mini ul>li:first-child>a,.pagination-small ul>li:first-child>a,.pagination-mini ul>li:first-child>span,.pagination-small ul>li:first-child>span{-webkit-border-bottom-left-radius:3px;border-bottom-left-radius:3px;-webkit-border-top-left-radius:3px;border-top-left-radius:3px;-moz-border-radius-bottomleft:3px;-moz-border-radius-topleft:3px}.pagination-mini ul>li:last-child>a,.pagination-small ul>li:last-child>a,.pagination-mini ul>li:last-child>span,.pagination-small ul>li:last-child>span{-webkit-border-top-right-radius:3px;border-top-right-radius:3px;-webkit-border-bottom-right-radius:3px;border-bottom-right-radius:3px;-moz-border-radius-topright:3px;-moz-border-radius-bottomright:3px}.pagination-small ul>li>a,.pagination-small ul>li>span{padding:2px 10px;font-size:11.9px}.pagination-mini ul>li>a,.pagination-mini ul>li>span{padding:0 6px;font-size:10.5px}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager li>a,.pager li>span{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#f5f5f5}.pager .next>a,.pager .next>span{float:right}.pager .previous>a,.pager .previous>span{float:left}.pager .disabled>a,.pager .disabled>a:hover,.pager .disabled>a:focus,.pager .disabled>span{color:#999;cursor:default;background-color:#fff}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:10%;left:50%;z-index:1050;width:560px;margin-left:-280px;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;outline:0;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:10%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{position:relative;max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.modal-footer .btn-block+.btn-block{margin-left:0}.tooltip{position:absolute;z-index:1030;display:block;font-size:11px;line-height:1.4;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{padding:5px 0;margin-top:-3px}.tooltip.right{padding:0 5px;margin-left:3px}.tooltip.bottom{padding:5px 0;margin-top:3px}.tooltip.left{padding:0 5px;margin-left:-3px}.tooltip-inner{max-width:200px;padding:8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;max-width:276px;padding:1px;text-align:left;white-space:normal;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-top:-10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-left:-10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-title:empty{display:none}.popover-content{padding:9px 14px}.popover .arrow,.popover .arrow:after{position:absolute;display:block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow{border-width:11px}.popover .arrow:after{border-width:10px;content:""}.popover.top .arrow{bottom:-11px;left:50%;margin-left:-11px;border-top-color:#999;border-top-color:rgba(0,0,0,0.25);border-bottom-width:0}.popover.top .arrow:after{bottom:1px;margin-left:-10px;border-top-color:#fff;border-bottom-width:0}.popover.right .arrow{top:50%;left:-11px;margin-top:-11px;border-right-color:#999;border-right-color:rgba(0,0,0,0.25);border-left-width:0}.popover.right .arrow:after{bottom:-10px;left:1px;border-right-color:#fff;border-left-width:0}.popover.bottom .arrow{top:-11px;left:50%;margin-left:-11px;border-bottom-color:#999;border-bottom-color:rgba(0,0,0,0.25);border-top-width:0}.popover.bottom .arrow:after{top:1px;margin-left:-10px;border-bottom-color:#fff;border-top-width:0}.popover.left .arrow{top:50%;right:-11px;margin-top:-11px;border-left-color:#999;border-left-color:rgba(0,0,0,0.25);border-right-width:0}.popover.left .arrow:after{right:1px;bottom:-10px;border-left-color:#fff;border-right-width:0}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover,a.thumbnail:focus{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.media,.media-body{overflow:hidden;*overflow:visible;zoom:1}.media,.media .media{margin-top:15px}.media:first-child{margin-top:0}.media-object{display:block}.media-heading{margin:0 0 5px}.media>.pull-left{margin-right:10px}.media>.pull-right{margin-left:10px}.media-list{margin-left:0;list-style:none}.label,.badge{display:inline-block;padding:2px 4px;font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding-right:9px;padding-left:9px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}.label:empty,.badge:empty{display:none}a.label:hover,a.label:focus,a.badge:hover,a.badge:focus{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel-inner>.item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel-inner>.item>img,.carousel-inner>.item>a>img{display:block;line-height:1}.carousel-inner>.active,.carousel-inner>.next,.carousel-inner>.prev{display:block}.carousel-inner>.active{left:0}.carousel-inner>.next,.carousel-inner>.prev{position:absolute;top:0;width:100%}.carousel-inner>.next{left:100%}.carousel-inner>.prev{left:-100%}.carousel-inner>.next.left,.carousel-inner>.prev.right{left:0}.carousel-inner>.active.left{left:-100%}.carousel-inner>.active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover,.carousel-control:focus{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-indicators{position:absolute;top:15px;right:15px;z-index:5;margin:0;list-style:none}.carousel-indicators li{display:block;float:left;width:10px;height:10px;margin-left:5px;text-indent:-999px;background-color:#ccc;background-color:rgba(255,255,255,0.25);border-radius:5px}.carousel-indicators .active{background-color:#fff}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;font-size:18px;font-weight:200;line-height:30px;color:inherit;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit li{line-height:30px}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css new file mode 100644 index 0000000000..c1ad4dbb17 --- /dev/null +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -0,0 +1,41 @@ +.navbar .brand { + height: 50px; + width: 110px; + margin-left: 1px; + padding: 0; +} + +.version { + line-height: 30px; + vertical-align: bottom; + font-size: 12px; + padding: 0; + margin: 0; + font-weight: bold; + color: #777; +} + +.navbar-inner { + padding-top: 2px; + height: 50px; +} + +.navbar-inner .nav { + margin-top: 5px; + font-size: 15px; +} + + +#infolist { + margin-left: 400px; + margin-top: 14px; +} + +#infolist li { + display: inline; + list-style-type: none; + list-style-position: outside; + padding-right: 20px; + padding-top: 10px; + padding-bottom: 10px; +} diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index fb2c803a91..33a16b5d84 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -77,7 +77,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executorTable}
    ; - UIUtils.sparkPage(content, "Application Info: " + app.desc.name) + UIUtils.basicSparkPage(content, "Application Info: " + app.desc.name) } def executorRow(executor: ExecutorInfo): Seq[Node] = { diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 4892f258e1..2ed566a5bc 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -79,7 +79,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { {completedAppsTable}
    ; - UIUtils.sparkPage(content, "Spark Master: " + state.uri) + UIUtils.basicSparkPage(content, "Spark Master: " + state.uri) } def workerRow(worker: WorkerInfo): Seq[Node] = { diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index e95a74a37f..c65d5b4faf 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -70,7 +70,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { ; - UIUtils.sparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) } def executorRow(executor: ExecutorRunner): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index fde3606740..5e20f41501 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -107,35 +107,76 @@ private[spark] object JettyUtils extends Logging { } } +object Page extends Enumeration { val Storage, Jobs = Value } + /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils { + import Page._ - /** Returns a page containing the supplied content and the spark web ui headers */ - def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String): Seq[Node] = { - val newContent = -
    -
    -
    - + /** Returns a spark page with correctly formatted headers */ + def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) + : Seq[Node] = { + val storage = page match { + case Storage =>
  • Storage
  • + case _ =>
  • Storage
  • + } + val jobs = page match { + case Jobs =>
  • Jobs
  • + case _ =>
  • Jobs
  • + } + + + + + + + + + {title} + + + +
    + +
    +
    + +
    + +
    +
    +

    + {title} +

    +
    +
    +
    + {content}
    -
    -
      -
    • Master: {sc.master}
    • -
    • Application: {sc.appName}
    • -
    • Executors: {sc.getExecutorStorageStatus.size}
    • -
    -
    -
    -
    ; - sparkPage(newContent ++ content, title) + + } - /** Returns a page containing the supplied content and the spark css, js, and logo. */ - def sparkPage(content: => Seq[Node], title: String): Seq[Node] = { + /** Returns a page with the spark css/js and a simple format. Used for scheduler UI. */ + def basicSparkPage(content: => Seq[Node], title: String): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 9b34cdd27f..f5ed08744c 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -11,6 +11,7 @@ import scala.xml.{NodeSeq, Node} import spark.scheduler.Stage import spark.ui.UIUtils._ +import spark.ui.Page._ import spark.storage.StorageLevel /** Page showing list of all ongoing and recently finished stages */ @@ -33,7 +34,7 @@ private[spark] class IndexPage(parent: JobProgressUI) {

    Completed Stages

    ++ completedStageTable ++

    Failed Stages

    ++ failedStageTable - headerSparkPage(content, parent.sc, "Spark Stages") + headerSparkPage(content, parent.sc, "Spark Stages", Jobs) } def getElapsedTime(submitted: Option[Long], completed: Long): String = { diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 518f48cb81..c9294a7261 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -7,6 +7,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import spark.ui.UIUtils._ +import spark.ui.Page._ import spark.util.Distribution import spark.Utils import spark.scheduler.cluster.TaskInfo @@ -26,7 +27,7 @@ private[spark] class StagePage(parent: JobProgressUI) {

    Summary Metrics

    No tasks have finished yet

    Tasks

    No tasks have finished yet
    - return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId)) + return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } val tasks = listener.stageToTaskInfos(stageId) @@ -70,7 +71,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val content =

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; - headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId)) + headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 772c669a1a..d284134391 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -7,6 +7,7 @@ import scala.xml.Node import spark.storage.{RDDInfo, StorageUtils} import spark.Utils import spark.ui.UIUtils._ +import spark.ui.Page._ /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { @@ -42,7 +43,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) {
    ++ {rddTable}; - headerSparkPage(content, parent.sc, "Spark Storage ") + headerSparkPage(content, parent.sc, "Spark Storage ", Storage) } def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index dd9fd2e4c5..65952f711a 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -8,6 +8,7 @@ import spark.storage.{StorageStatus, StorageUtils} import spark.ui.UIUtils._ import spark.Utils import spark.storage.BlockManagerMasterActor.BlockStatus +import spark.ui.Page._ /** Page showing storage details for a given RDD */ private[spark] class RDDPage(parent: BlockManagerUI) { @@ -70,7 +71,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) { ; - headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name) + headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name, Jobs) } def blockRow(blk: (String, BlockStatus)): Seq[Node] = { From 62c2c6b8561a00b2103fe5fb80b42b9975a5b8db Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 21:47:22 -0700 Subject: [PATCH 065/419] Forcing Jetty to run as daemon --- core/src/main/scala/spark/ui/JettyUtils.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 5e20f41501..8cc50bc5b0 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -14,6 +14,7 @@ import scala.xml.Node import spark.{SparkContext, Logging} import org.eclipse.jetty.util.log.Log +import org.eclipse.jetty.util.thread.QueuedThreadPool /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { @@ -93,9 +94,15 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { val server = new Server(currentPort) + val pool = new QueuedThreadPool + pool.setDaemon(true) + server.setThreadPool(pool) server.setHandler(handlerList) + Try { server.start() } match { - case s: Success[_] => (server, server.getConnectors.head.getLocalPort) + case s: Success[_] => + sys.addShutdownHook(server.stop()) // Be kind, un-bind + (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) @@ -103,6 +110,8 @@ private[spark] object JettyUtils extends Logging { connect((currentPort + 1) % 65536) } } + + connect(port) } } From c537e869f3a12e75fac862cc6fdcf27a3e1e01b4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 22:02:03 -0700 Subject: [PATCH 066/419] Missing logo file --- .../spark/ui/static/spark-logo-77x50px-hd.png | Bin 0 -> 3536 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png diff --git a/core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png b/core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png new file mode 100644 index 0000000000000000000000000000000000000000..6c5f0993c43f0d9e1e1a04391d8a9157f0fdfba8 GIT binary patch literal 3536 zcmV;>4KMPEP)4Tx05}naRo`#hR1`jmZ&IWdKOk5~hl<6oRa0BJ8yc;~21%2p?MfD<>DVeH z9(p*dx19w`~g7O0}n_%Aq@s%d)fBDv`JHkDym6Hd+5XuAtvnwRpGmK zVkc9?T=n|PIo~X-eVh__(Z?q}P9Z-Dj?gOW6|D%o20XmjW-qs4UjrD(li^iv8@eK9k+ZFm zVRFymFOPAzG5-%Pn|1W;U4vNroTa&AxDScmEA~{ri9gr1^c?U@uwSpaNnw8l_>cP1 zd;)kMQS_;jeRSUEM_*s96y65j1$)tOrwdK{YIQMt92l|D^(E_=$Rjw{b!QT@q!)ni zR`|5oW9X5n$Wv+HVc@|^eX5yXnsHX8PF3UX~a6)MwxDE0HaPjyrlI!;jX{6Kvuh*8ej?;85ekN$?5uuCiS zBTvvVG+XTxAO{m@bvM#Jr)z6J><&E22D|vq?Y?Vkbo_DijopiF$2PET#mZ8eu=y$(ArYkv7@Ex`GL?QCc!_*KFrd&;n1r7 zqW-CFs9&fT)ZaU5gc&=gBz-DaCw(vdOp0__x+47~U6sC(E(JNe@4cTT*n6*E zVH4eoU1-&7pEV~_PRe`a7v+@vy!^5}8?Y3)UmlaER00009a7bBm000XU000XU0RWnu7ytkXQb|NXRA>e5 zSRhOo=*hmAER0R0YUQ5&?|AZ?L`ST`WM6a>^FLsRE~`yXWtl?w+2db2a{%eVhd4JE8MirO_aSq+gq*ou-ZU=tMS$2;L~N>BV;{wvcP<7Z_^#Xj=fhGn zif=t*TpK?89DUkm5wUAvW9vl!o*TLbli^DTS`@^q*4Y};iIMBs7&6=iN$VNr&8F$Q zeF7+21l#uUaYC;V zR6w;;vZ>ARVX2PzR~H40(fs;pcjB_RthZ<291>Y;0`Izl!WlITbxo_lI`?&3=ri-Q zdrM3BoZ|dR7T)zchbJeD0l%5zgZ99lU_ohXlQoXqb6|EQkc%K)^YM<$;~?fE;l1%) zcQtND#8gYWyL{{M-8)MsE@uvh&K~gh;o3s?-kqiGm76!GEigU)|GmvX&Llr~)cVWG@3)*vA*QX@6--0RtTDtPQ6}8;?wwlz#6MCK zis=Rh+j;Tocp^xq^?r@Eq#%F$_ZMdWU%#ti4P~wa0B*i`^=3!bw7k3=En)mAD{|HE2!z z;Wk>Q0D!d@-k;w5$I+X1te7IWpg%3&s-)QGYO?>Q#!i57Ur*XMg^+EIi}-v1l7N=2 zjhUD@m+ss+Rc4$d8v;1 zYBDA=_}HfZrO9=oT>a}qG(cDXi93OfW z`SI|h^~GyPJ0$P!?WH}glXO&?@V6!}475XGPSkEJOI!pzF)l!NVEjVXz53$#xTE=` zQ)ZqPPqgXUaP?I!@S6Zwnm2r8X;!|5hdBpHiygYE#>4dKB$n=fA< zH{v!3|7p8`FC#;{hC?hY!iS>sqsqjfNxZ z_Z+_mtOaX|N(NveI;)-ljf+7Zc9NI?qY$`(&96vY$gs-J(B{`gE)!2$gKII~_!vf) zj0(p`KweK@FzI=y2Wu^jV(2(~*N4F&nF(M#q)!>;69D6AiI_%)H+{Gx7ss{~g6Oa4 zhygc#Br@v4ZC2BOQ68CpD7t{(n>>U5mOv3>l?01l!uPo9e=jh~dJ#2H~EYB%0A_<@(%v0D?Aw1w&ZxILfi zcER+6bap=xbBFeZ?xSfO&?h|wGeP47Tm(;FPevNq@fsJc`Ouqva(49KkNSjB9Yi(4 zS`rlDPpdt^NJ>Qx(0f8l4R;NM9@Va;+Ch|dDAvc5Q$gbpesQC>Ts$JLb7ahwnDG$= zbGNbD_#6G0cYaEx73|zej{r9gG;i^JzmWvjR_Qfb=VNIPn$LHTzu-3PlGdG}(AMpA zE=L(&3u{_O)4U-WhlGbuFB4Xo&fG;eu9L1N0F_0awI;{{kprwY!v|kD!t=E=+fk@V zu+M7bxGCP2CQ2+nEG}$ICjtmSwxrFPn@(>MploEFT1H9{O1goeLXO9R{uQq~YQ}## zCi&Xc#9H)0VC{d8JAs%6#3(r*yD7)emzy4A7F{3!sRh)LtCe{=cn($3IZy+H3-3-_ z0J)}kp2TfNORX_JVaz{ZO!T#Z7e;?_*2O1{JZcxWSbRXt*D!AldDy;JA6ai1fIPui zdYsPaU2Ef^;i_TW0Ku3$L`Nw)X3|Zu{)*ajnFqt%PxQ~Kb1>Q645cy#-2gw(GH#k? z4p~L>h4y@GKC3URrJ--g=o3vx`bKr&liiVZm&uF5LlDV%Y5Y%UshP+M&3YmoQNnCD zwOgZuP>M6ioHtn{doy%+W=K3on@-0(%4YS2ECkrA$}MAc=5}bLfs5rNV2uH)xDdqj zMI>J@L@Aw)q~mhhP1z5|vzOkRjF+zcR}LniFwwU-#u^j?r4#~`7G46olDPJ-7s?Dm zQW#u)A}Q_^3DB<%wD+^XlD<)Hx8mcWGc|9iy1zjxGq4>z%i7Y%4Q$WmeLReLvNq9S z?dOje?4xC*hqkbpbA9O9qR39K`*75h_OyE98AU~!s&*yaIsH6=wUnaGYqd*F!*VCf zG{C5texA%5V8kF3Gzz8_-i%5%Gaf0uOGTa=H0-`C2q11u#0rsJ<~lQ`UFHq{5_`Ld zrTr4~e%51Lfp&r8y8`VBvD@(foXg>qSyzO9lVzjpa0000< KMNUMnLSTaP)3PW4 literal 0 HcmV?d00001 From 249f0e54ba6c358b3d786b1ba42e5d2ea22ed9db Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 28 Jun 2013 13:25:26 -0700 Subject: [PATCH 067/419] Minor changes from Matei's review --- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- .../main/scala/spark/ui/jobs/IndexPage.scala | 21 ++++++++----------- .../scala/spark/ui/jobs/JobProgressUI.scala | 2 +- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 8cc50bc5b0..9e4b884379 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -160,8 +160,8 @@ private[spark] object UIUtils { {jobs}
      -
    • Master: {sc.master}
    • Application: {sc.appName}
    • +
    • Master: {sc.master}
    • Executors: {sc.getExecutorStorageStatus.size}
    diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index f5ed08744c..27c017d7ee 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -1,7 +1,5 @@ package spark.ui.jobs -import akka.util.Duration - import java.util.Date import javax.servlet.http.HttpServletRequest @@ -26,9 +24,9 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow(), activeStages) - val completedStageTable = listingTable(stageHeaders, stageRow(), completedStages) - val failedStageTable: NodeSeq = listingTable(stageHeaders, stageRow(false), failedStages) + val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow, activeStages) + val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) + val failedStageTable: NodeSeq = listingTable(stageHeaders, stageRow, failedStages) val content =

    Active Stages

    ++ activeStageTable ++

    Completed Stages

    ++ completedStageTable ++ @@ -44,7 +42,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { } } - def makeSlider(completed: Int, total: Int): Seq[Node] = { + def makeProgressBar(completed: Int, total: Int): Seq[Node] = { val width=130 val height=15 val completeWidth = (completed.toDouble / total) * width @@ -53,12 +51,12 @@ private[spark] class IndexPage(parent: JobProgressUI) { + fill="rgb(51,51,51)" stroke="black" stroke-width="1" /> } - def stageRow(showLink: Boolean = true)(s: Stage): Seq[Node] = { + def stageRow(s: Stage): Seq[Node] = { val submissionTime = s.submissionTime match { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" @@ -74,13 +72,12 @@ private[spark] class IndexPage(parent: JobProgressUI) { val totalTasks = s.numPartitions - {if (showLink) {{s.id}} - else {{s.id}}} - {s.origin} + {s.id} + {s.origin} {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeSlider(completedTasks, totalTasks)} + {makeProgressBar(completedTasks, totalTasks)} {completedTasks} / {totalTasks} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index a18bf0f81e..f584d1e187 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -23,7 +23,7 @@ import spark.Utils private[spark] class JobProgressUI(val sc: SparkContext) { private var _listener: Option[JobProgressListener] = None def listener = _listener.get - val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") + val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) From 8113c55df8f4b5f34140ddba5e58e132e3dc2d23 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Fri, 28 Jun 2013 13:46:21 -0700 Subject: [PATCH 068/419] [Feedback] Get rid of -m, set MASTER from SPARK_MASTER_IP/PORT automagically --- spark-shell | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/spark-shell b/spark-shell index ea67a3e6b8..a8e72143fb 100755 --- a/spark-shell +++ b/spark-shell @@ -1,24 +1,14 @@ #!/bin/bash --posix # # Shell script for starting the Spark Shell REPL +# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} +# if those two env vars are set in spark-env.sh but MASTER is not. # Options: -# -m Set MASTER to spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT # -c Set the number of cores for REPL to use # FWDIR="`dirname $0`" for o in "$@"; do - if [ "$1" = "-m" -o "$1" = "--master" ]; then - shift - if [ -e "$FWDIR/conf/spark-env.sh" ]; then - . "$FWDIR/conf/spark-env.sh" - fi - if [ -z "$MASTER" ]; then - MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - fi - export MASTER - fi - if [ "$1" = "-c" -o "$1" = "--cores" ]; then shift if [ -n "$1" ]; then @@ -28,6 +18,17 @@ for o in "$@"; do fi done +# Set MASTER from spark-env if possible +if [ -z "$MASTER" ]; then + if [ -e "$FWDIR/conf/spark-env.sh" ]; then + . "$FWDIR/conf/spark-env.sh" + fi + if [[ "x" != "x$SPARK_MASTER_IP" && "y" != "y$SPARK_MASTER_PORT" ]]; then + MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" + export MASTER + fi +fi + # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed exit_status=127 From 473961d82e70f4cdb9e3d12b0dfcb4b70e6121ab Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 29 Jun 2013 08:38:04 -0700 Subject: [PATCH 069/419] Styling for progress bar --- .../main/resources/spark/ui/static/webui.css | 8 ++++++ .../main/scala/spark/ui/jobs/IndexPage.scala | 28 +++++++++++++++---- 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css index c1ad4dbb17..f7537bb766 100644 --- a/core/src/main/resources/spark/ui/static/webui.css +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -39,3 +39,11 @@ padding-top: 10px; padding-bottom: 10px; } + +.progress-cell { + width: 134px; + border-right: 0; + padding: 0; + padding-top: 7px; + padding-left: 4px; +} diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 27c017d7ee..cd04542faa 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -24,9 +24,27 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow, activeStages) - val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) - val failedStageTable: NodeSeq = listingTable(stageHeaders, stageRow, failedStages) + /** Special table which merges two header cells. */ + def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + + + + + + + + + + + {rows.map(r => makeRow(r))} + +
    Stage IdOriginSubmittedDurationTasks: Complete/TotalShuffle ActivityStored RDD
    + } + + val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) + val completedStageTable = stageTable(stageRow, completedStages) + val failedStageTable: NodeSeq = stageTable(stageRow, failedStages) val content =

    Active Stages

    ++ activeStageTable ++

    Completed Stages

    ++ completedStageTable ++ @@ -77,8 +95,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeProgressBar(completedTasks, totalTasks)} - {completedTasks} / {totalTasks} + {makeProgressBar(completedTasks, totalTasks)} + {completedTasks} / {totalTasks} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) case _ => From e721ff7e5a2d738750c40e95d6ba53898eaa7051 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 29 Jun 2013 11:26:30 -0700 Subject: [PATCH 070/419] Allowing details for failed stages --- .../scala/spark/scheduler/DAGScheduler.scala | 6 +- .../cluster/ClusterTaskSetManager.scala | 1 + .../scheduler/local/LocalTaskSetManager.scala | 1 + .../scala/spark/ui/jobs/JobProgressUI.scala | 30 ++++---- .../main/scala/spark/ui/jobs/StagePage.scala | 77 +++++++++++-------- 5 files changed, 69 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 82d419453b..d9ddc41aa2 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -618,8 +618,11 @@ class DAGScheduler( handleExecutorLost(bmAddress.executorId, Some(task.generation)) } + case ExceptionFailure(className, description, stackTrace) => + // Do nothing here, left up to the TaskScheduler to decide how to handle user failures + case other => - // Non-fetch failure -- probably a bug in user code; abort all jobs depending on this stage + // Unrecognized failure - abort all jobs depending on this stage abortStage(idToStage(task.stageId), task + " failed: " + other) } } @@ -667,6 +670,7 @@ class DAGScheduler( */ private def abortStage(failedStage: Stage, reason: String) { val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq + failedStage.completionTime = Some(System.currentTimeMillis()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) val error = new SparkException("Job failed: " + reason) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index d72b0bfc9f..6965cde5da 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -571,6 +571,7 @@ private[spark] class ClusterTaskSetManager( return case ef: ExceptionFailure => + sched.listener.taskEnded(tasks(index), ef, null, null, info, null) val key = ef.description val now = System.currentTimeMillis val (printFull, dupCount) = { diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index 70b69bb26f..499116f653 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -152,6 +152,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas info.markFailed() decreaseRunningTasks(1) val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) + sched.listener.taskEnded(task, reason, null, null, info, null) if (!finished(index)) { copiesRunning(index) -= 1 numFailures(index) += 1 diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index f584d1e187..aafa414055 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -12,12 +12,11 @@ import scala.Seq import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} import spark.ui.JettyUtils._ -import spark.SparkContext +import spark.{ExceptionFailure, SparkContext, Success, Utils} import spark.scheduler._ import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics -import spark.Success -import spark.Utils +import collection.mutable /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { @@ -51,7 +50,8 @@ private[spark] class JobProgressListener extends SparkListener { val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() - val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() + val stageToTaskInfos = + HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]]() override def onJobStart(jobStart: SparkListenerJobStart) {} @@ -67,8 +67,6 @@ private[spark] class JobProgressListener extends SparkListener { if (stages.size > RETAINED_STAGES) { val toRemove = RETAINED_STAGES / 10 stages.takeRight(toRemove).foreach( s => { - stageToTasksComplete.remove(s.id) - stageToTasksFailed.remove(s.id) stageToTaskInfos.remove(s.id) }) stages.trimEnd(toRemove) @@ -80,14 +78,18 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - taskEnd.reason match { - case Success => - stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - case _ => - stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - } - val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) - taskList += ((taskEnd.taskInfo, taskEnd.taskMetrics)) + val failureInfo: Option[ExceptionFailure] = + taskEnd.reason match { + case e: ExceptionFailure => + stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 + Some(e) + case _ => + stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 + None + } + val taskList = stageToTaskInfos.getOrElse( + sid, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]()) + taskList += ((taskEnd.taskInfo, taskEnd.taskMetrics, failureInfo)) stageToTaskInfos(sid) = taskList } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index c9294a7261..ed96fc2994 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -9,7 +9,7 @@ import scala.xml.Node import spark.ui.UIUtils._ import spark.ui.Page._ import spark.util.Distribution -import spark.Utils +import spark.{ExceptionFailure, Utils} import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics @@ -38,56 +38,71 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskHeaders: Seq[String] = Seq("Task ID", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ - {if (shuffleWrite) Seq("Shuffle Write") else Nil} + {if (shuffleWrite) Seq("Shuffle Write") else Nil} ++ + Seq("Details") val taskTable = listingTable(taskHeaders, taskRow, tasks) - val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} - val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( - ms => parent.formatDuration(ms.toLong)) + // Excludes tasks which failed and have incomplete metrics + val validTasks = tasks.filter(t => Option(t._2).isDefined) - def getQuantileCols(data: Seq[Double]) = - Distribution(data).get.getQuantiles().map(d => Utils.memoryBytesToString(d.toLong)) + val summaryTable: Option[Seq[Node]] = + if (validTasks.size == 0) { + None + } + else { + val serviceTimes = validTasks.map{case (info, metrics, exception) => + metrics.executorRunTime.toDouble} + val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( + ms => parent.formatDuration(ms.toLong)) - val shuffleReadSizes = tasks.map { - case(info, metrics) => - metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble - } - val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) + def getQuantileCols(data: Seq[Double]) = + Distribution(data).get.getQuantiles().map(d => Utils.memoryBytesToString(d.toLong)) - val shuffleWriteSizes = tasks.map { - case(info, metrics) => - metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble - } - val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) + val shuffleReadSizes = validTasks.map { + case(info, metrics, exception) => + metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + } + val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) - val listings: Seq[Seq[String]] = Seq(serviceQuantiles, - if (shuffleRead) shuffleReadQuantiles else Nil, - if (shuffleWrite) shuffleWriteQuantiles else Nil) + val shuffleWriteSizes = validTasks.map { + case(info, metrics, exception) => + metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + } + val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) - val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") - val quantileTable = listingTable(quantileHeaders, quantileRow, listings) + val listings: Seq[Seq[String]] = Seq(serviceQuantiles, + if (shuffleRead) shuffleReadQuantiles else Nil, + if (shuffleWrite) shuffleWriteQuantiles else Nil) + + val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") + def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} + Some(listingTable(quantileHeaders, quantileRow, listings)) + } val content = -

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; +

    Summary Metrics

    ++ summaryTable.getOrElse(Nil) ++

    Tasks

    ++ taskTable; headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } - def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} - def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { - val (info, metrics) = taskData + def taskRow(taskData: (TaskInfo, TaskMetrics, Option[ExceptionFailure])): Seq[Node] = { + def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = + trace.map(e => {e.toString}) + val (info, metrics, exception) = taskData {info.taskId} - {parent.formatDuration(metrics.executorRunTime)} + {Option(metrics).map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} - {metrics.shuffleReadMetrics.map{m => - {Utils.memoryBytesToString(m.remoteBytesRead)}}.getOrElse("") } - {metrics.shuffleWriteMetrics.map{m => - {Utils.memoryBytesToString(m.shuffleBytesWritten)}}.getOrElse("") } + {Option(metrics).flatMap{m => m.shuffleReadMetrics}.map{s => + {Utils.memoryBytesToString(s.remoteBytesRead)}}.getOrElse("")} + {Option(metrics).flatMap{m => m.shuffleWriteMetrics}.map{s => + {Utils.memoryBytesToString(s.shuffleBytesWritten)}}.getOrElse("")} + {exception.map(e => + {e.className}
    {fmtStackTrace(e.stackTrace)}
    ).getOrElse("")} } } From 4358acfe07e991090fbe009aafe3f5110fbf0c40 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 29 Jun 2013 15:25:06 -0700 Subject: [PATCH 071/419] Initialize Twitter4J OAuth from system properties instead of prompting --- .../spark/streaming/StreamingContext.scala | 4 ++- .../api/java/JavaStreamingContext.scala | 23 ++++++------- .../dstream/TwitterInputDStream.scala | 32 ++++++------------- .../java/spark/streaming/JavaAPISuite.java | 2 +- 4 files changed, 23 insertions(+), 38 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index e61438fe3a..36b841af8f 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -381,7 +381,9 @@ class StreamingContext private ( /** * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J authentication + * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth + * authorization; this uses the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret. * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index c4a223b419..ed7b789d98 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -307,7 +307,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization + * @param twitterAuth Twitter4J Authorization object * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ @@ -320,10 +320,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Create a input stream that returns tweets received from Twitter using - * java.util.Preferences to store OAuth token. OAuth key and secret should - * be provided using system properties twitter4j.oauth.consumerKey and - * twitter4j.oauth.consumerSecret + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret to be set. * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ @@ -347,10 +346,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Create a input stream that returns tweets received from Twitter using - * java.util.Preferences to store OAuth token. OAuth key and secret should - * be provided using system properties twitter4j.oauth.consumerKey and - * twitter4j.oauth.consumerSecret + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret to be set. * @param filters Set of filter strings to get only those tweets that match them */ def twitterStream( @@ -370,10 +368,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Create a input stream that returns tweets received from Twitter using - * java.util.Preferences to store OAuth token. OAuth key and secret should - * be provided using system properties twitter4j.oauth.consumerKey and - * twitter4j.oauth.consumerSecret + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret to be set. */ def twitterStream(): JavaDStream[Status] = { ssc.twitterStream() diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index e0c654d385..ff7a58be45 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -4,21 +4,21 @@ import spark._ import spark.streaming._ import storage.StorageLevel import twitter4j._ -import twitter4j.auth.BasicAuthorization import twitter4j.auth.Authorization import java.util.prefs.Preferences +import twitter4j.conf.ConfigurationBuilder import twitter4j.conf.PropertyConfiguration import twitter4j.auth.OAuthAuthorization import twitter4j.auth.AccessToken /* A stream of Twitter statuses, potentially filtered by one or more keywords. * -* @constructor create a new Twitter stream using the supplied username and password to authenticate. +* @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials. * An optional set of string filters can be used to restrict the set of tweets. The Twitter API is * such that this may return a sampled subset of all tweets during each interval. * -* Includes a simple implementation of OAuth using consumer key and secret provided using system -* properties twitter4j.oauth.consumerKey and twitter4j.oauth.consumerSecret +* If no Authorization object is provided, initializes OAuth authorization using the system +* properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. */ private[streaming] class TwitterInputDStream( @@ -28,28 +28,14 @@ class TwitterInputDStream( storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - lazy val createOAuthAuthorization: Authorization = { - val userRoot = Preferences.userRoot(); - val token = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN, null)) - val tokenSecret = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, null)) - val oAuth = new OAuthAuthorization(new PropertyConfiguration(System.getProperties())) - if (token.isEmpty || tokenSecret.isEmpty) { - val requestToken = oAuth.getOAuthRequestToken() - println("Authorize application using URL: "+requestToken.getAuthorizationURL()) - println("Enter PIN: ") - val pin = Console.readLine - val accessToken = if (pin.length() > 0) oAuth.getOAuthAccessToken(requestToken, pin) else oAuth.getOAuthAccessToken() - userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN, accessToken.getToken()) - userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, accessToken.getTokenSecret()) - userRoot.flush() - } else { - oAuth.setOAuthAccessToken(new AccessToken(token.get, tokenSecret.get)); - } - oAuth + private def createOAuthAuthorization(): Authorization = { + new OAuthAuthorization(new ConfigurationBuilder().build()) } + + private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(if (twitterAuth.isEmpty) createOAuthAuthorization else twitterAuth.get, filters, storageLevel) + new TwitterReceiver(authorization, filters, storageLevel) } } diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index e5fdbe1b7a..4cf10582a9 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -1267,7 +1267,7 @@ public class JavaAPISuite implements Serializable { @Test public void testTwitterStream() { String[] filters = new String[] { "good", "bad", "ugly" }; - JavaDStream test = ssc.twitterStream("username", "password", filters, StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY()); } @Test From 5cfcd3c336cc13e9fd448ae122216e4b583b77b4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 29 Jun 2013 15:37:27 -0700 Subject: [PATCH 072/419] Remove Twitter4J specific repo since it's in Maven central --- pom.xml | 11 ----------- project/SparkBuild.scala | 3 +-- 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/pom.xml b/pom.xml index 3bcb2a3f34..7a31be98b2 100644 --- a/pom.xml +++ b/pom.xml @@ -109,17 +109,6 @@ false - - twitter4j-repo - Twitter4J Repository - http://twitter4j.org/maven2/ - - true - - - false - - diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 07572201de..5e4692162e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -138,8 +138,7 @@ object SparkBuild extends Build { resolvers ++= Seq( "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "Spray Repository" at "http://repo.spray.cc/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", - "Twitter4J Repository" at "http://twitter4j.org/maven2/" + "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), libraryDependencies ++= Seq( From 03d0b858c807339b4221bedffa29ac76eef5352e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 15:38:58 -0700 Subject: [PATCH 073/419] Made use of spark.executor.memory setting consistent and documented it Conflicts: core/src/main/scala/spark/SparkContext.scala --- core/src/main/scala/spark/SparkContext.scala | 24 +++++++++----- .../scheduler/cluster/SchedulerBackend.scala | 11 ++----- docs/configuration.md | 31 ++++++++++++------- docs/ec2-scripts.md | 5 ++- docs/tuning.md | 6 ++-- 5 files changed, 43 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 70a9d7698c..366afb2a2a 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -115,13 +115,14 @@ class SparkContext( // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner - for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", - "SPARK_TESTING")) { + for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) { val value = System.getenv(key) if (value != null) { executorEnvs(key) = value } } + // Since memory can be set with a system property too, use that + executorEnvs("SPARK_MEM") = SparkContext.executorMemoryRequested + "m" if (environment != null) { executorEnvs ++= environment } @@ -156,14 +157,12 @@ class SparkContext( scheduler case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => - // Check to make sure SPARK_MEM <= memoryPerSlave. Otherwise Spark will just hang. + // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. val memoryPerSlaveInt = memoryPerSlave.toInt - val sparkMemEnv = System.getenv("SPARK_MEM") - val sparkMemEnvInt = if (sparkMemEnv != null) Utils.memoryStringToMb(sparkMemEnv) else 512 - if (sparkMemEnvInt > memoryPerSlaveInt) { + if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) { throw new SparkException( - "Slave memory (%d MB) cannot be smaller than SPARK_MEM (%d MB)".format( - memoryPerSlaveInt, sparkMemEnvInt)) + "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format( + memoryPerSlaveInt, SparkContext.executorMemoryRequested)) } val scheduler = new ClusterScheduler(this) @@ -881,6 +880,15 @@ object SparkContext { /** Find the JAR that contains the class of a particular object */ def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) + + /** Get the amount of memory per executor requested through system properties or SPARK_MEM */ + private[spark] val executorMemoryRequested = { + // TODO: Might need to add some extra memory for the non-heap parts of the JVM + Option(System.getProperty("spark.executor.memory")) + .orElse(Option(System.getenv("SPARK_MEM"))) + .map(Utils.memoryStringToMb) + .getOrElse(512) + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala index 9ac875de3a..8844057a5c 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala @@ -1,6 +1,6 @@ package spark.scheduler.cluster -import spark.Utils +import spark.{SparkContext, Utils} /** * A backend interface for cluster scheduling systems that allows plugging in different ones under @@ -14,14 +14,7 @@ private[spark] trait SchedulerBackend { def defaultParallelism(): Int // Memory used by each executor (in megabytes) - protected val executorMemory = { - // TODO: Might need to add some extra memory for the non-heap parts of the JVM - Option(System.getProperty("spark.executor.memory")) - .orElse(Option(System.getenv("SPARK_MEM"))) - .map(Utils.memoryStringToMb) - .getOrElse(512) - } - + protected val executorMemory: Int = SparkContext.executorMemoryRequested // TODO: Probably want to add a killTask too } diff --git a/docs/configuration.md b/docs/configuration.md index 2de512f896..ae61769e31 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -25,23 +25,25 @@ Inside `spark-env.sh`, you *must* set at least the following two variables: * `SCALA_HOME`, to point to your Scala installation. * `MESOS_NATIVE_LIBRARY`, if you are [running on a Mesos cluster](running-on-mesos.html). -In addition, there are four other variables that control execution. These can be set *either in `spark-env.sh` -or in each job's driver program*, because they will automatically be propagated to workers from the driver. -For a multi-user environment, we recommend setting the in the driver program instead of `spark-env.sh`, so -that different user jobs can use different amounts of memory, JVM options, etc. +In addition, there are four other variables that control execution. These should be set *in the environment that +launches the job's driver program* instead of `spark-env.sh`, because they will be automatically propagated to +workers. Setting these per-job instead of in `spark-env.sh` ensures that different jobs can have different settings +for these variables. -* `SPARK_MEM`, to set the amount of memory used per node (this should be in the same format as the - JVM's -Xmx option, e.g. `300m` or `1g`) * `SPARK_JAVA_OPTS`, to add JVM options. This includes any system properties that you'd like to pass with `-D`. * `SPARK_CLASSPATH`, to add elements to Spark's classpath. * `SPARK_LIBRARY_PATH`, to add search directories for native libraries. +* `SPARK_MEM`, to set the amount of memory used per node. This should be in the same format as the + JVM's -Xmx option, e.g. `300m` or `1g`. Note that this option will soon be deprecated in favor of + the `spark.executor.memory` system property, so we recommend using that in new code. -Note that if you do set these in `spark-env.sh`, they will override the values set by user programs, which -is undesirable; you can choose to have `spark-env.sh` set them only if the user program hasn't, as follows: +Beware that if you do set these variables in `spark-env.sh`, they will override the values set by user programs, +which is undesirable; if you prefer, you can choose to have `spark-env.sh` set them only if the user program +hasn't, as follows: {% highlight bash %} -if [ -z "$SPARK_MEM" ] ; then - SPARK_MEM="1g" +if [ -z "$SPARK_JAVA_OPTS" ] ; then + SPARK_JAVA_OPTS="-verbose:gc" fi {% endhighlight %} @@ -55,10 +57,17 @@ val sc = new SparkContext(...) {% endhighlight %} Most of the configurable system properties control internal settings that have reasonable default values. However, -there are at least four properties that you will commonly want to control: +there are at least five properties that you will commonly want to control: + + + + + diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index dc57035eba..eab8a0ff20 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -106,9 +106,8 @@ permissions on your private key file, you can run `launch` with the # Configuration You can edit `/root/spark/conf/spark-env.sh` on each machine to set Spark configuration options, such -as JVM options and, most crucially, the amount of memory to use per machine (`SPARK_MEM`). -This file needs to be copied to **every machine** to reflect the change. The easiest way to do this -is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, +as JVM options. This file needs to be copied to **every machine** to reflect the change. The easiest way to +do this is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, then run `~/spark-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers. The [configuration guide](configuration.html) describes the available configuration options. diff --git a/docs/tuning.md b/docs/tuning.md index 32c7ab86e9..5ffca54481 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -157,9 +157,9 @@ their work directories), *not* on your driver program. **Cache Size Tuning** -One important configuration parameter for GC is the amount of memory that should be used for -caching RDDs. By default, Spark uses 66% of the configured memory (`SPARK_MEM`) to cache RDDs. This means that - 33% of memory is available for any objects created during task execution. +One important configuration parameter for GC is the amount of memory that should be used for caching RDDs. +By default, Spark uses 66% of the configured executor memory (`spark.executor.memory` or `SPARK_MEM`) to +cache RDDs. This means that 33% of memory is available for any objects created during task execution. In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of memory, lowering this value will help reduce the memory consumption. To change this to say 50%, you can call From 5bbd0eec84867937713ceb8438f25a943765a084 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 17:00:26 -0700 Subject: [PATCH 074/419] Update docs on SCALA_LIBRARY_PATH --- conf/spark-env.sh.template | 18 ++++++------------ docs/configuration.md | 4 +++- 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 37565ca827..b8936314ec 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -3,8 +3,10 @@ # This file contains environment variables required to run Spark. Copy it as # spark-env.sh and edit that to configure Spark for your site. At a minimum, # the following two variables should be set: -# - MESOS_NATIVE_LIBRARY, to point to your Mesos native library (libmesos.so) -# - SCALA_HOME, to point to your Scala installation +# - SCALA_HOME, to point to your Scala installation, or SCALA_LIBRARY_PATH to +# point to the directory for Scala library JARs (if you install Scala as a +# Debian or RPM package, these are in a separate path, often /usr/share/java) +# - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # # If using the standalone deploy mode, you can also set variables for it: # - SPARK_MASTER_IP, to bind the master to a different IP address @@ -12,14 +14,6 @@ # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT -# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes to be spawned on every slave machine -# -# Finally, Spark also relies on the following variables, but these can be set -# on just the *master* (i.e. in your driver program), and will automatically -# be propagated to workers: -# - SPARK_MEM, to change the amount of memory used per node (this should -# be in the same format as the JVM's -Xmx option, e.g. 300m or 1g) -# - SPARK_CLASSPATH, to add elements to Spark's classpath -# - SPARK_JAVA_OPTS, to add JVM options -# - SPARK_LIBRARY_PATH, to add extra search paths for native libraries. +# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes +# to be spawned on every slave machine diff --git a/docs/configuration.md b/docs/configuration.md index ae61769e31..3266db7af1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -22,7 +22,9 @@ the copy executable. Inside `spark-env.sh`, you *must* set at least the following two variables: -* `SCALA_HOME`, to point to your Scala installation. +* `SCALA_HOME`, to point to your Scala installation, or `SCALA_LIBRARY_PATH` to point to the directory for Scala + library JARs (if you install Scala as a Debian or RPM package, there is no `SCALA_HOME`, but these libraries + are in a separate path, typically /usr/share/java; look for `scala-library.jar`). * `MESOS_NATIVE_LIBRARY`, if you are [running on a Mesos cluster](running-on-mesos.html). In addition, there are four other variables that control execution. These should be set *in the environment that From 39ae073b5cd0dcfe4a00d9f205c88bad9df37870 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 17:11:14 -0700 Subject: [PATCH 075/419] Increase SLF4j version in Maven too --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7a31be98b2..48e623fa1c 100644 --- a/pom.xml +++ b/pom.xml @@ -56,7 +56,7 @@ 2.0.3 1.0-M2.1 1.1.1 - 1.6.1 + 1.7.2 4.1.2 1.2.17 From 3296d132b6ce042843de6e7384800e089b49e5fa Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 02:45:00 +0000 Subject: [PATCH 076/419] Fix performance bug with new Python code not using buffered streams --- core/src/main/scala/spark/SparkEnv.scala | 3 +- .../scala/spark/api/python/PythonRDD.scala | 33 ++++++++++--------- 2 files changed, 19 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7ccde2e818..ec59b4f48f 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -59,7 +59,8 @@ class SparkEnv ( def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = { synchronized { - pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create() + val key = (pythonExec, envVars) + pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 63140cf37f..3f283afa62 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -45,37 +45,38 @@ private[spark] class PythonRDD[T: ClassManifest]( new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) - val out = new PrintWriter(worker.getOutputStream) - val dOut = new DataOutputStream(worker.getOutputStream) + val stream = new BufferedOutputStream(worker.getOutputStream) + val dataOut = new DataOutputStream(stream) + val printOut = new PrintWriter(stream) // Partition index - dOut.writeInt(split.index) + dataOut.writeInt(split.index) // sparkFilesDir - PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dOut) + PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) // Broadcast variables - dOut.writeInt(broadcastVars.length) + dataOut.writeInt(broadcastVars.length) for (broadcast <- broadcastVars) { - dOut.writeLong(broadcast.id) - dOut.writeInt(broadcast.value.length) - dOut.write(broadcast.value) - dOut.flush() + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) } + dataOut.flush() // Serialized user code for (elem <- command) { - out.println(elem) + printOut.println(elem) } - out.flush() + printOut.flush() // Data values for (elem <- parent.iterator(split, context)) { - PythonRDD.writeAsPickle(elem, dOut) + PythonRDD.writeAsPickle(elem, dataOut) } - dOut.flush() - out.flush() + dataOut.flush() + printOut.flush() worker.shutdownOutput() } }.start() // Return an iterator that read lines from the process's stdout - val stream = new DataInputStream(worker.getInputStream) + val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream)) return new Iterator[Array[Byte]] { def next(): Array[Byte] = { val obj = _nextObj @@ -288,7 +289,7 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) // This happens on the master, where we pass the updates to Python through a socket val socket = new Socket(serverHost, serverPort) val in = socket.getInputStream - val out = new DataOutputStream(socket.getOutputStream) + val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream)) out.writeInt(val2.size) for (array <- val2) { out.writeInt(array.length) From ec31e68d5df259e6df001529235d8c906ff02a6f Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 06:20:14 +0000 Subject: [PATCH 077/419] Fixed PySpark perf regression by not using socket.makefile(), and improved debuggability by letting "print" statements show up in the executor's stderr Conflicts: core/src/main/scala/spark/api/python/PythonRDD.scala --- .../scala/spark/api/python/PythonRDD.scala | 10 +++-- .../api/python/PythonWorkerFactory.scala | 20 ++++++++- python/pyspark/daemon.py | 42 +++++++++++-------- 3 files changed, 50 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 3f283afa62..31d8ea89d4 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -22,6 +22,8 @@ private[spark] class PythonRDD[T: ClassManifest]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) def this(parent: RDD[T], command: String, envVars: JMap[String, String], @@ -45,7 +47,7 @@ private[spark] class PythonRDD[T: ClassManifest]( new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) - val stream = new BufferedOutputStream(worker.getOutputStream) + val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) val dataOut = new DataOutputStream(stream) val printOut = new PrintWriter(stream) // Partition index @@ -76,7 +78,7 @@ private[spark] class PythonRDD[T: ClassManifest]( }.start() // Return an iterator that read lines from the process's stdout - val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream)) + val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) return new Iterator[Array[Byte]] { def next(): Array[Byte] = { val obj = _nextObj @@ -276,6 +278,8 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { Utils.checkHost(serverHost, "Expected hostname") + + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList @@ -289,7 +293,7 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) // This happens on the master, where we pass the updates to Python through a socket val socket = new Socket(serverHost, serverPort) val in = socket.getInputStream - val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream)) + val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream, bufferSize)) out.writeInt(val2.size) for (array <- val2) { out.writeInt(array.length) diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 8844411d73..85d1dfeac8 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -51,7 +51,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val workerEnv = pb.environment() workerEnv.putAll(envVars) daemon = pb.start() - daemonPort = new DataInputStream(daemon.getInputStream).readInt() // Redirect the stderr to ours new Thread("stderr reader for " + pythonExec) { @@ -69,6 +68,25 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } } }.start() + + val in = new DataInputStream(daemon.getInputStream) + daemonPort = in.readInt() + + // Redirect further stdout output to our stderr + new Thread("stdout reader for " + pythonExec) { + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME HACK: We copy the stream on the level of bytes to + // attempt to dodge encoding problems. + var buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + System.err.write(buf, 0, len) + len = in.read(buf) + } + } + } + }.start() } catch { case e => { stopDaemon() diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 78a2da1e18..78c9457b84 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -1,10 +1,13 @@ import os +import signal +import socket import sys +import traceback import multiprocessing from ctypes import c_bool from errno import EINTR, ECHILD -from socket import socket, AF_INET, SOCK_STREAM, SOMAXCONN -from signal import signal, SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN +from socket import AF_INET, SOCK_STREAM, SOMAXCONN +from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN from pyspark.worker import main as worker_main from pyspark.serializers import write_int @@ -33,11 +36,12 @@ def compute_real_exit_code(exit_code): def worker(listen_sock): # Redirect stdout to stderr os.dup2(2, 1) + sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 # Manager sends SIGHUP to request termination of workers in the pool def handle_sighup(*args): assert should_exit() - signal(SIGHUP, handle_sighup) + signal.signal(SIGHUP, handle_sighup) # Cleanup zombie children def handle_sigchld(*args): @@ -51,7 +55,7 @@ def worker(listen_sock): handle_sigchld() elif err.errno != ECHILD: raise - signal(SIGCHLD, handle_sigchld) + signal.signal(SIGCHLD, handle_sigchld) # Handle clients while not should_exit(): @@ -70,19 +74,22 @@ def worker(listen_sock): # never receives SIGCHLD unless a worker crashes. if os.fork() == 0: # Leave the worker pool - signal(SIGHUP, SIG_DFL) + signal.signal(SIGHUP, SIG_DFL) listen_sock.close() - # Handle the client then exit - sockfile = sock.makefile() + # Read the socket using fdopen instead of socket.makefile() because the latter + # seems to be very slow; note that we need to dup() the file descriptor because + # otherwise writes also cause a seek that makes us miss data on the read side. + infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) exit_code = 0 try: - worker_main(sockfile, sockfile) + worker_main(infile, outfile) except SystemExit as exc: - exit_code = exc.code + exit_code = exc.code finally: - sockfile.close() - sock.close() - os._exit(compute_real_exit_code(exit_code)) + outfile.flush() + sock.close() + os._exit(compute_real_exit_code(exit_code)) else: sock.close() @@ -92,7 +99,6 @@ def launch_worker(listen_sock): try: worker(listen_sock) except Exception as err: - import traceback traceback.print_exc() os._exit(1) else: @@ -105,7 +111,7 @@ def manager(): os.setpgid(0, 0) # Create a listening socket on the AF_INET loopback interface - listen_sock = socket(AF_INET, SOCK_STREAM) + listen_sock = socket.socket(AF_INET, SOCK_STREAM) listen_sock.bind(('127.0.0.1', 0)) listen_sock.listen(max(1024, 2 * POOLSIZE, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() @@ -121,8 +127,8 @@ def manager(): exit_flag.value = True # Gracefully exit on SIGTERM, don't die on SIGHUP - signal(SIGTERM, lambda signum, frame: shutdown()) - signal(SIGHUP, SIG_IGN) + signal.signal(SIGTERM, lambda signum, frame: shutdown()) + signal.signal(SIGHUP, SIG_IGN) # Cleanup zombie children def handle_sigchld(*args): @@ -133,7 +139,7 @@ def manager(): except EnvironmentError as err: if err.errno not in (ECHILD, EINTR): raise - signal(SIGCHLD, handle_sigchld) + signal.signal(SIGCHLD, handle_sigchld) # Initialization complete sys.stdout.close() @@ -148,7 +154,7 @@ def manager(): shutdown() raise finally: - signal(SIGTERM, SIG_DFL) + signal.signal(SIGTERM, SIG_DFL) exit_flag.value = True # Send SIGHUP to notify workers of shutdown os.kill(0, SIGHUP) From 7cd490ef5ba28df31f5e061eff83c855731dfca4 Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 06:25:17 +0000 Subject: [PATCH 078/419] Clarify that PySpark is not supported on Windows --- docs/python-programming-guide.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 7f1e7cf93d..e8aaac74d0 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -17,10 +17,9 @@ There are a few key differences between the Python and Scala APIs: * Python is dynamically typed, so RDDs can hold objects of different types. * PySpark does not currently support the following Spark features: - Special functions on RDDs of doubles, such as `mean` and `stdev` - - `lookup` + - `lookup`, `sample` and `sort` - `persist` at storage levels other than `MEMORY_ONLY` - - `sample` - - `sort` + - Execution on Windows -- this is slated for a future release In PySpark, RDDs support the same methods as their Scala counterparts but take Python functions and return Python collection types. Short functions can be passed to RDD methods using Python's [`lambda`](http://www.diveintopython.net/power_of_introspection/lambda_functions.html) syntax: From 5de326db7d5235dbf259a4c8b388cb24c40fd2ec Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 09:19:45 -0700 Subject: [PATCH 079/419] Print exception message --- core/src/main/scala/spark/scheduler/SparkListener.scala | 1 + core/src/main/scala/spark/ui/jobs/StagePage.scala | 6 +++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index bac984b5c9..8de3aa91a4 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -68,6 +68,7 @@ class StatsReportListener extends SparkListener with Logging { showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) //runtime breakdown + val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ case (info, metrics) => RuntimePercentage(info.duration, metrics) } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index ed96fc2994..c6f87fc652 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -102,7 +102,11 @@ private[spark] class StagePage(parent: JobProgressUI) { {Option(metrics).flatMap{m => m.shuffleWriteMetrics}.map{s => }.getOrElse("")} + + {e.className} ({e.description})
    + {fmtStackTrace(e.stackTrace)} +
    ).getOrElse("")} + } } From 735c951a09f71512cf7be834802d1e88567ca9ae Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 09:33:22 -0700 Subject: [PATCH 080/419] Adding test script --- .../scala/spark/ui/UIWorkloadGenerator.scala | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 core/src/main/scala/spark/ui/UIWorkloadGenerator.scala diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala new file mode 100644 index 0000000000..24cfe36aaa --- /dev/null +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -0,0 +1,61 @@ +package spark.ui + +import scala.util.Random + +import spark.SparkContext +import spark.SparkContext._ + +/** + * Continuously generates jobs that expose various features of the WebUI (internal testing tool). + * + * Usage: ./run spark.ui.UIWorkloadGenerator [master] + */ +private[spark] object UIWorkloadGenerator { + val NUM_PARTITIONS = 100 + val INTER_JOB_WAIT_MS = 500 + + def main(args: Array[String]) { + val master = args(0) + val appName = "Spark UI Tester" + val sc = new SparkContext(master, appName) + + // NOTE: Right now there is no easy way for us to show spark.job.annotation for a given phase, + // but we pass it here anyways since it will be useful once we do. + def setName(s: String) = { + sc.addLocalProperties("spark.job.annotation", s) + } + val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) + def nextFloat() = (new Random()).nextFloat() + + val jobs = Seq[(String, () => Long)]( + ("Count", baseData.count), + ("Cache and Count", baseData.map(x => x).cache.count), + ("Single Shuffle", baseData.map(x => (x % 10, x)).reduceByKey(_ + _).count), + ("Entirely failed phase", baseData.map(x => throw new Exception).count), + ("Partially failed phase", { + baseData.map{x => + val probFailure = (4.0 / NUM_PARTITIONS) + if (nextFloat() < probFailure) { + throw new Exception("This is a task failure") + } + 1 + }.count + }), + ("Job with delays", baseData.map(x => Thread.sleep(1000)).count) + ) + + while (true) { + for ((desc, job) <- jobs) { + try { + setName(desc) + job() + println("Job funished: " + desc) + } catch { + case e: Exception => + println("Job Failed: " + desc) + } + Thread.sleep(INTER_JOB_WAIT_MS) + } + } + } +} From 8688689387eb6e2ac34add5e460dc54fe32b6def Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 13:40:12 -0700 Subject: [PATCH 081/419] Various formatting changes --- core/src/main/scala/spark/Utils.scala | 8 ++++---- .../src/main/scala/spark/ui/jobs/IndexPage.scala | 6 ++---- core/src/test/scala/spark/ui/UISuite.scala | 16 ++++++++-------- 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 6966ee9ee9..af926b1e36 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -495,13 +495,13 @@ private object Utils extends Logging { ms match { case t if t < second => - "%dms".format(t) + "%d ms".format(t) case t if t < minute => - "%d.%03ds".format(t / second, t % second) + "%.1f s".format(t.toFloat / second) case t if t < hour => - "%d:%02d".format(t / minute, (t % minute) / second) + "%.1f m".format(t.toFloat / minute) case t => - "%d:%02d:%02d".format(t / hour, t % hour / minute, (t % hour) % minute / second) + "%.2f h".format(t.toFloat / hour) } } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index cd04542faa..1e675ab2cb 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -18,8 +18,6 @@ private[spark] class IndexPage(parent: JobProgressUI) { val dateFmt = parent.dateFmt def render(request: HttpServletRequest): Seq[Node] = { - val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Progress", - "Tasks: Complete/Total", "Shuffle Activity", "Stored RDD") val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq @@ -67,9 +65,9 @@ private[spark] class IndexPage(parent: JobProgressUI) { + fill="white" stroke="rgb(51,51,51)" stroke-width="1" /> + fill="rgb(0,136,204)" stroke="black" stroke-width="1" /> } diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 127ab5ebc2..aa191f628b 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -34,13 +34,13 @@ class UISuite extends FunSuite { val hour = minute * 60 def str = Utils.msDurationToString(_) - assert(str(123) === "123ms") - assert(str(second) === "1.000s") - assert(str(second + 452) === "1.452s") - assert(str(hour) === "1:00:00") - assert(str(minute) === "1:00") - assert(str(minute + 4 * second + 34) === "1:04") - assert(str(10 * hour + minute + 4 * second) === "10:01:04") - assert(str(10 * hour + 59 * minute + 59 * second + 999) === "10:59:59") + assert(str(123) === "123 ms") + assert(str(second) === "1.0 s") + assert(str(second + 462) === "1.5 s") + assert(str(hour) === "1.00 h") + assert(str(minute) === "1.0 m") + assert(str(minute + 4 * second + 34) === "1.1 m") + assert(str(10 * hour + minute + 4 * second) === "10.02 h") + assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") } } From 30b90342410a196d99d25113fbc1e37ce2a8200f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 13:48:01 -0700 Subject: [PATCH 082/419] Fixing bug where logs aren't shown --- core/src/main/scala/spark/deploy/worker/Worker.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index b6a26245fc..6ae1cef940 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -45,7 +45,7 @@ private[spark] class Worker( val envVar = System.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - val webUi = new WorkerWebUI(self, workDir, Some(webUiPort)) + var webUi: WorkerWebUI = null var coresUsed = 0 var memoryUsed = 0 @@ -77,6 +77,7 @@ private[spark] class Worker( sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() + webUi = new WorkerWebUI(self, workDir, Some(webUiPort)) webUi.start() connectToMaster() } From 1025d7d1efb20e3683a604259c92eeef400cc3fa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 14:40:53 -0700 Subject: [PATCH 083/419] Package refactoring --- core/src/main/scala/spark/ui/JettyUtils.scala | 116 +----------------- core/src/main/scala/spark/ui/Page.scala | 3 + core/src/main/scala/spark/ui/UIUtils.scala | 113 +++++++++++++++++ 3 files changed, 118 insertions(+), 114 deletions(-) create mode 100644 core/src/main/scala/spark/ui/Page.scala create mode 100644 core/src/main/scala/spark/ui/UIUtils.scala diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 9e4b884379..57f85bafd2 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -8,13 +8,12 @@ import net.liftweb.json.{JValue, pretty, render} import org.eclipse.jetty.server.{Server, Request, Handler} import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} +import org.eclipse.jetty.util.thread.QueuedThreadPool import scala.util.{Try, Success, Failure} import scala.xml.Node -import spark.{SparkContext, Logging} -import org.eclipse.jetty.util.log.Log -import org.eclipse.jetty.util.thread.QueuedThreadPool +import spark.Logging /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { @@ -111,117 +110,6 @@ private[spark] object JettyUtils extends Logging { } } - connect(port) } } - -object Page extends Enumeration { val Storage, Jobs = Value } - -/** Utility functions for generating XML pages with spark content. */ -private[spark] object UIUtils { - import Page._ - - /** Returns a spark page with correctly formatted headers */ - def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) - : Seq[Node] = { - val storage = page match { - case Storage =>
  • Storage
  • - case _ =>
  • Storage
  • - } - val jobs = page match { - case Jobs =>
  • Jobs
  • - case _ =>
  • Jobs
  • - } - - - - - - - - - {title} - - - -
    - -
    -
    - -
    -
    - -
    -
    -

    - {title} -

    -
    -
    -
    - {content} -
    - - - } - - /** Returns a page with the spark css/js and a simple format. Used for scheduler UI. */ - def basicSparkPage(content: => Seq[Node], title: String): Seq[Node] = { - - - - - - - {title} - - - -
    -
    -
    - -
    -
    -

    - {title} -

    -
    -
    - {content} -
    - - - } - - /** Returns an HTML table constructed by generating a row for each object in a sequence. */ - def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { -
    Property NameDefaultMeaning
    spark.executor.memory512m + Amount of memory to use per executor process, in the same format as JVM memory strings (e.g. `512m`, `2g`). +
    spark.serializer spark.JavaSerializer{Utils.memoryBytesToString(s.shuffleBytesWritten)}{exception.map(e => - {e.className}
    {fmtStackTrace(e.stackTrace)}
    ).getOrElse("")}
    - {headers.map(h => )} - - {rows.map(r => makeRow(r))} - -
    {h}
    - } -} diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala new file mode 100644 index 0000000000..c853b44b76 --- /dev/null +++ b/core/src/main/scala/spark/ui/Page.scala @@ -0,0 +1,3 @@ +package spark.ui + +private[spark] object Page extends Enumeration { val Storage, Jobs = Value } \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala new file mode 100644 index 0000000000..7b79290d1b --- /dev/null +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -0,0 +1,113 @@ +package spark.ui + +import scala.xml.Node + +import spark.SparkContext + +/** Utility functions for generating XML pages with spark content. */ +private[spark] object UIUtils { + import Page._ + + /** Returns a spark page with correctly formatted headers */ + def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) + : Seq[Node] = { + val storage = page match { + case Storage =>
  • Storage
  • + case _ =>
  • Storage
  • + } + val jobs = page match { + case Jobs =>
  • Jobs
  • + case _ =>
  • Jobs
  • + } + + + + + + + + + {title} + + + +
    + +
    +
    + +
    +
    + +
    +
    +

    + {title} +

    +
    +
    +
    + {content} +
    + + + } + + /** Returns a page with the spark css/js and a simple format. Used for scheduler UI. */ + def basicSparkPage(content: => Seq[Node], title: String): Seq[Node] = { + + + + + + + {title} + + + +
    +
    +
    + +
    +
    +

    + {title} +

    +
    +
    + {content} +
    + + + } + + /** Returns an HTML table constructed by generating a row for each object in a sequence. */ + def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + {headers.map(h => )} + + {rows.map(r => makeRow(r))} + +
    {h}
    + } +} \ No newline at end of file From 9a42d04efa1ff620306ec8864c9f95186fc1a199 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 14:43:13 -0700 Subject: [PATCH 084/419] Throw exception for missing resource --- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 57f85bafd2..bc6f9c10d5 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -69,7 +69,7 @@ private[spark] object JettyUtils extends Logging { case Some(res) => staticHandler.setResourceBase(res.toString) case None => - logError("Could not find resource path for Web UI: " + resourceBase) + throw new Exception("Could not find resource path for Web UI: " + resourceBase) } staticHandler } From 6fdbc68f2c5b220d1618d5a78d46aa0f844cae45 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Mon, 1 Jul 2013 16:05:55 -0700 Subject: [PATCH 085/419] Fixing missed hbase dependency in examples hadoop2-yarn profile --- examples/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/examples/pom.xml b/examples/pom.xml index 3e5271ec2f..78ec58729b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -193,6 +193,11 @@ hadoop-yarn-common provided + + org.apache.hbase + hbase + 0.94.6 + From 8ca1cc1786a376b227ba69872efbf8e26952b80d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 2 Jul 2013 15:56:34 -0700 Subject: [PATCH 086/419] Adding truncation for log files --- core/src/main/scala/spark/Utils.scala | 14 +++++++++ .../spark/deploy/worker/ui/WorkerWebUI.scala | 16 +++++----- core/src/test/scala/spark/ui/UISuite.scala | 31 +++++++++++++++++++ 3 files changed, 54 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index af926b1e36..c1616de641 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -603,6 +603,20 @@ private object Utils extends Logging { portBound } + /** Return a string containing the last `n` bytes of a file. */ + def lastNBytes(path: String, n: Int): String = { + val file = new File(path) + val length = file.length() + val buff = new Array[Byte](math.min(n, length.toInt)) + val skip = math.max(0, length - n) + val stream = new FileInputStream(file) + + stream.skip(skip) + stream.read(buff) + stream.close() + Source.fromBytes(buff).mkString + } + /** * Clone an object using a Spark serializer. */ diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 2e6279566c..16564d5619 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -3,14 +3,12 @@ package spark.deploy.worker.ui import akka.actor.ActorRef import akka.util.{Duration, Timeout} -import java.io.File +import java.io.{FileInputStream, File} import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import scala.io.Source - import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ @@ -56,11 +54,15 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") + + val maxBytes = 1024 * 1024 // Guard against OOM + val defaultBytes = 100 * 1024 + val numBytes = Option(request.getParameter("numBytes")) + .flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val source = Source.fromFile(path) - val lines = source.mkString - source.close() - lines + val pre = "==== Last %s bytes of %s/%s/%s ====\n".format(numBytes, appId, executorId, logType) + pre + Utils.lastNBytes(path, math.min(numBytes, maxBytes)) } def stop() { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index aa191f628b..e4bb3abc33 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -5,6 +5,10 @@ import org.eclipse.jetty.server.Server import java.net.ServerSocket import scala.util.{Failure, Success, Try} import spark.Utils +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import java.io.{FileOutputStream, File} +import com.google.common.base.Charsets class UISuite extends FunSuite { test("jetty port increases under contention") { @@ -43,4 +47,31 @@ class UISuite extends FunSuite { assert(str(10 * hour + minute + 4 * second) === "10.02 h") assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") } + + test("reading last n bytes of a file") { + val tmpDir = Files.createTempDir() + + // File smaller than limit + val f1Path = tmpDir + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) + f1.close() + assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") + + // File larger than limit + val f2Path = tmpDir + "/f2" + val f2 = new FileOutputStream(f2Path) + f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f2.close() + assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") + + // Request limit too + val f3Path = tmpDir + "/f2" + val f3 = new FileOutputStream(f3Path) + f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f3.close() + assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") + + FileUtils.deleteDirectory(tmpDir) + } } From 39e2325675e40c2231e26a7e693035f54039b1d5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 2 Jul 2013 16:28:40 -0700 Subject: [PATCH 087/419] Removing dead code --- core/src/main/scala/spark/Utils.scala | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c1616de641..c5faf79ac3 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -589,19 +589,6 @@ private object Utils extends Logging { "%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile, callSiteInfo.firstUserLine) } - /** - * Try to find a free port to bind to on the local host. This should ideally never be needed, - * except that, unfortunately, some of the networking libraries we currently rely on (e.g. Spray) - * don't let users bind to port 0 and then figure out which free port they actually bound to. - * We work around this by binding a ServerSocket and immediately unbinding it. This is *not* - * necessarily guaranteed to work, but it's the best we can do. - */ - def findFreePort(): Int = { - val socket = new ServerSocket(0) - val portBound = socket.getLocalPort - socket.close() - portBound - } /** Return a string containing the last `n` bytes of a file. */ def lastNBytes(path: String, n: Int): String = { From 923cf929003c67963e273fcdcd5b01baf68df8b5 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Tue, 2 Jul 2013 21:18:59 -0500 Subject: [PATCH 088/419] Rework from pull request. Removed --user option from Spark on Yarn Client, made the user of JAVA_HOME environment variable conditional on if its set, and created addCredentials in each of the SparkHadoopUtil classes to only add the credentials when the profile is hadoop2-yarn. --- .../scala/spark/deploy/SparkHadoopUtil.scala | 4 ++++ .../scala/spark/deploy/SparkHadoopUtil.scala | 7 +++++++ .../scala/spark/deploy/yarn/Client.scala | 14 +++++++++----- .../scala/spark/deploy/yarn/ClientArguments.scala | 8 +------- .../scala/spark/deploy/yarn/WorkerRunnable.scala | 12 +++++++----- .../scala/spark/deploy/SparkHadoopUtil.scala | 4 ++++ core/src/main/scala/spark/PairRDDFunctions.scala | 4 ---- core/src/main/scala/spark/SparkContext.scala | 8 -------- core/src/main/scala/spark/rdd/HadoopRDD.scala | 2 ++ .../scala/spark/scheduler/InputFormatInfo.scala | 11 +++-------- .../spark/streaming/PairDStreamFunctions.scala | 4 ---- 11 files changed, 37 insertions(+), 41 deletions(-) diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index a0fb4fe25d..f1c86de4cc 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,5 +1,6 @@ package spark.deploy import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.JobConf /** @@ -20,4 +21,7 @@ object SparkHadoopUtil { // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems def newConfiguration(): Configuration = new Configuration() + + // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + def addCredentials(conf: JobConf) {} } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala index b96c047e10..301a57fffa 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,6 +1,7 @@ package spark.deploy import collection.mutable.HashMap +import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.conf.Configuration @@ -49,4 +50,10 @@ object SparkHadoopUtil { // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems // Always create a new config, dont reuse yarnConf. def newConfiguration(): Configuration = new YarnConfiguration(new Configuration()) + + // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + def addCredentials(conf: JobConf) { + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) + } } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala index f20cc31c7c..514c17f241 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -45,7 +45,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl appContext.setQueue(args.amQueue) appContext.setAMContainerSpec(amContainer) - appContext.setUser(args.amUser) + appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) submitApp(appContext) @@ -141,9 +141,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val log4jConfLocalRes = localResources.getOrElse("log4j.properties", null) val env = new HashMap[String, String]() - Apps.addToEnvironment(env, Environment.USER.name, args.amUser) - // set this so that UGI set to correct user in unsecure mode - Apps.addToEnvironment(env, "HADOOP_USER_NAME", args.amUser) // If log4j present, ensure ours overrides all others if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./") @@ -171,6 +168,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl env("SPARK_YARN_LOG4J_SIZE") = log4jConfLocalRes.getSize().toString() } + // Add each SPARK-* key to the environment System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } return env @@ -224,7 +222,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } // Command for the ApplicationMaster - val commands = List[String](Environment.JAVA_HOME.$() + "/bin/java " + + var javaCommand = "java"; + val javaHome = System.getenv("JAVA_HOME") + if (javaHome != null && !javaHome.isEmpty()) { + javaCommand = Environment.JAVA_HOME.$() + "/bin/java" + } + + val commands = List[String](javaCommand + " -server " + JAVA_OPTS + " spark.deploy.yarn.ApplicationMaster" + diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala index 24110558e7..07e7edea36 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala @@ -13,7 +13,6 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 var workerCores = 1 var numWorkers = 2 - var amUser = System.getProperty("user.name") var amQueue = System.getProperty("QUEUE", "default") var amMemory: Int = 512 // TODO @@ -58,10 +57,6 @@ class ClientArguments(val args: Array[String]) { workerCores = value args = tail - case ("--user") :: value :: tail => - amUser = value - args = tail - case ("--queue") :: value :: tail => amQueue = value args = tail @@ -96,8 +91,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" + - " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + - " --user USERNAME Run the ApplicationMaster (and slaves) as a different user\n" + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')" ) System.exit(exitCode) } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala index e22d256a84..cc6f3344a1 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -85,7 +85,13 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S credentials.writeTokenStorageToStream(dob) ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - val commands = List[String](Environment.JAVA_HOME.$() + "/bin/java " + + var javaCommand = "java"; + val javaHome = System.getenv("JAVA_HOME") + if (javaHome != null && !javaHome.isEmpty()) { + javaCommand = Environment.JAVA_HOME.$() + "/bin/java" + } + + val commands = List[String](javaCommand + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state. @@ -152,10 +158,6 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() - // should we add this ? - Apps.addToEnvironment(env, Environment.USER.name, Utils.getUserNameFromEnvironment()) - // set this so that UGI set to correct user in unsecure mode - Apps.addToEnvironment(env, "HADOOP_USER_NAME", Utils.getUserNameFromEnvironment()) // If log4j present, ensure ours overrides all others if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) { diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index a0fb4fe25d..f1c86de4cc 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,5 +1,6 @@ package spark.deploy import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.JobConf /** @@ -20,4 +21,7 @@ object SparkHadoopUtil { // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems def newConfiguration(): Configuration = new Configuration() + + // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + def addCredentials(conf: JobConf) {} } diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index b397601184..c9d698fc09 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -614,10 +614,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf(self.context.hadoopConfiguration), codec: Option[Class[_ <: CompressionCodec]] = None) { - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) conf.setOutputKeyClass(keyClass) conf.setOutputValueClass(valueClass) // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index c3a56938b5..6c37203707 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -295,10 +295,6 @@ class SparkContext( valueClass: Class[V], minSplits: Int = defaultMinSplits ): RDD[(K, V)] = { - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) } @@ -311,10 +307,6 @@ class SparkContext( minSplits: Int = defaultMinSplits ) : RDD[(K, V)] = { val conf = new JobConf(hadoopConfiguration) - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) FileInputFormat.setInputPaths(conf, path) new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) } diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index cbf5512e24..07c103503c 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -15,6 +15,7 @@ import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.util.ReflectionUtils +import spark.deploy.SparkHadoopUtil import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} import spark.util.NextIterator import org.apache.hadoop.conf.Configurable @@ -50,6 +51,7 @@ class HadoopRDD[K, V]( private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) override def getPartitions: Array[Partition] = { + SparkHadoopUtil.addCredentials(conf); val inputFormat = createInputFormat(conf) if (inputFormat.isInstanceOf[Configurable]) { inputFormat.asInstanceOf[Configurable].setConf(conf) diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala index 30a56d7135..17d0ea4f80 100644 --- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala @@ -9,6 +9,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.conf.Configuration import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ +import spark.deploy.SparkHadoopUtil /** @@ -71,10 +72,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // This method does not expect failures, since validate has already passed ... private def prefLocsFromMapreduceInputFormat(): Set[SplitInfo] = { val conf = new JobConf(configuration) - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) + SparkHadoopUtil.addCredentials(conf); FileInputFormat.setInputPaths(conf, path) val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] = @@ -94,10 +92,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // This method does not expect failures, since validate has already passed ... private def prefLocsFromMapredInputFormat(): Set[SplitInfo] = { val jobConf = new JobConf(configuration) - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = jobConf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) + SparkHadoopUtil.addCredentials(jobConf); FileInputFormat.setInputPaths(jobConf, path) val instance: org.apache.hadoop.mapred.InputFormat[_, _] = diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 20ee1d3c5d..8d0a83d439 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -472,10 +472,6 @@ extends Serializable { outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf ) { - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) val saveFunc = (rdd: RDD[(K, V)], time: Time) => { val file = rddToFileName(prefix, suffix, time) rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, conf) From 96130c30d90abf155731346488f79c034bcaaf6a Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Wed, 3 Jul 2013 05:49:04 +0000 Subject: [PATCH 089/419] add compression codec trait and snappy compression --- .../scala/spark/storage/BlockManager.scala | 20 +++++++++++++++++-- .../spark/storage/CompressionCodec.scala | 13 ++++++++++++ .../main/scala/spark/storage/DiskStore.scala | 1 - .../spark/storage/LZFCompressionCodec.scala | 16 +++++++++++++++ .../storage/SnappyCompressionCodec.scala | 18 +++++++++++++++++ 5 files changed, 65 insertions(+), 3 deletions(-) create mode 100644 core/src/main/scala/spark/storage/CompressionCodec.scala create mode 100644 core/src/main/scala/spark/storage/LZFCompressionCodec.scala create mode 100644 core/src/main/scala/spark/storage/SnappyCompressionCodec.scala diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9b39d3aadf..2d4a3502c6 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -141,6 +141,8 @@ private[spark] class BlockManager( val metadataCleaner = new MetadataCleaner("BlockManager", this.dropOldBlocks) initialize() + var compressionCodec: CompressionCodec = null + /** * Construct a BlockManager with a memory limit set based on system properties. */ @@ -902,8 +904,15 @@ private[spark] class BlockManager( * Wrap an output stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: OutputStream): OutputStream = { + if (compressionCodec == null) { + compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", + "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) + .newInstance().asInstanceOf[CompressionCodec] + } + if (shouldCompress(blockId)) { - (new LZFOutputStream(s)).setFinishBlockOnFlush(true) + //(new LZFOutputStream(s)).setFinishBlockOnFlush(true) + compressionCodec.compressionOutputStream(s) } else { s } @@ -913,7 +922,14 @@ private[spark] class BlockManager( * Wrap an input stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: InputStream): InputStream = { - if (shouldCompress(blockId)) new LZFInputStream(s) else s + if (compressionCodec == null) { + compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", + "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) + .newInstance().asInstanceOf[CompressionCodec] + } + + if (shouldCompress(blockId)) /*new LZFInputStream(s) */ + compressionCodec.compressionInputStream(s) else s } def dataSerialize( diff --git a/core/src/main/scala/spark/storage/CompressionCodec.scala b/core/src/main/scala/spark/storage/CompressionCodec.scala new file mode 100644 index 0000000000..cd80de33f6 --- /dev/null +++ b/core/src/main/scala/spark/storage/CompressionCodec.scala @@ -0,0 +1,13 @@ +package spark.storage + +import java.io.{InputStream, OutputStream} + + +/** + * CompressionCodec allows the customization of the compression codec + */ +trait CompressionCodec { + def compressionOutputStream(s: OutputStream): OutputStream + + def compressionInputStream(s: InputStream): InputStream +} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index da859eebcb..221e285192 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -49,7 +49,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def close() { if (initialized) { objOut.close() - bs.close() channel = null bs = null objOut = null diff --git a/core/src/main/scala/spark/storage/LZFCompressionCodec.scala b/core/src/main/scala/spark/storage/LZFCompressionCodec.scala new file mode 100644 index 0000000000..3328b949ef --- /dev/null +++ b/core/src/main/scala/spark/storage/LZFCompressionCodec.scala @@ -0,0 +1,16 @@ +package spark.storage + +import java.io.{InputStream, OutputStream} + +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} + +/** + * LZF implementation of [[spark.storage.CompressionCodec]] + */ +class LZFCompressionCodec extends CompressionCodec { + def compressionOutputStream(s: OutputStream): OutputStream = + (new LZFOutputStream(s)).setFinishBlockOnFlush(true) + + def compressionInputStream(s: InputStream): InputStream = + new LZFInputStream(s) +} diff --git a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala new file mode 100644 index 0000000000..dc8546b039 --- /dev/null +++ b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala @@ -0,0 +1,18 @@ +package spark.storage + +import java.io.{InputStream, OutputStream} + +import org.xerial.snappy.SnappyOutputStream + +/** + * Snappy implementation of [[spark.storage.CompressionCodec]] + * block size can be configured by spark.snappy.block.size + */ +class SnappyCompressionCodec extends CompressionCodec { + def compressionOutputStream(s: OutputStream): OutputStream = + new SnappyOutputStream(s, + System.getProperty("spark.snappy.block.size", "32768").toInt) + + def compressionInputStream(s: InputStream): InputStream = + new SnappyInputStream(s) +} From 04567a1771ec02e1efcc3ce078291a975fbb4e68 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Wed, 3 Jul 2013 17:43:37 +0800 Subject: [PATCH 090/419] update guava version from 11.0.1 to 14.0.1 --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5e4692162e..f65e398d95 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -142,7 +142,8 @@ object SparkBuild extends Build { ), libraryDependencies ++= Seq( - "com.google.guava" % "guava" % "11.0.1", + "com.google.guava" % "guava" % "14.0.1", + "com.google.code.findbugs" % "jsr305" % "1.3.+", "log4j" % "log4j" % "1.2.16", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, From 94238aae57475030f6e88102a83c7809c5835494 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Wed, 3 Jul 2013 18:08:38 +0000 Subject: [PATCH 091/419] fix dependencies --- core/src/main/scala/spark/storage/SnappyCompressionCodec.scala | 2 +- project/SparkBuild.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala index dc8546b039..62b00ef3f6 100644 --- a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala +++ b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala @@ -2,7 +2,7 @@ package spark.storage import java.io.{InputStream, OutputStream} -import org.xerial.snappy.SnappyOutputStream +import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} /** * Snappy implementation of [[spark.storage.CompressionCodec]] diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 07572201de..f824826af3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -162,7 +162,8 @@ object SparkBuild extends Build { "cc.spray" % "spray-json_2.9.2" % "1.1.1" excludeAll(excludeNetty), "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", - "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "org.apache.derby" % "derby" % "10.4.2.0" % "test", + "org.xerial.snappy" % "snappy-java" % "1.0.5" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { From 0f06d6217d0da67633d9ede73df67b94133fd3d5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jul 2013 01:05:39 -0700 Subject: [PATCH 092/419] s/ActorSystemImpl/ExtendedActorSystem/ as ActorSystemImpl results in a warning --- core/src/main/scala/spark/util/AkkaUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index e93cc3b485..0cff0d5b01 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,6 +1,6 @@ package spark.util -import akka.actor.{ActorRef, Props, ActorSystemImpl, ActorSystem} +import akka.actor.{ActorRef, Props, ExtendedActorSystem, ActorSystem} import com.typesafe.config.ConfigFactory import akka.util.duration._ import akka.pattern.ask @@ -56,7 +56,7 @@ private[spark] object AkkaUtils { // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. - val provider = actorSystem.asInstanceOf[ActorSystemImpl].provider + val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get return (actorSystem, boundPort) } From 6ccfb73ca92a72d5cca6a504d9ee332a16d9293d Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 4 Jul 2013 19:19:44 +0800 Subject: [PATCH 093/419] Add fair scheduler config template file --- conf/fairscheduler.xml.template | 15 +++++++++++++++ core/src/test/resources/fairscheduler.xml | 1 + 2 files changed, 16 insertions(+) create mode 100644 conf/fairscheduler.xml.template diff --git a/conf/fairscheduler.xml.template b/conf/fairscheduler.xml.template new file mode 100644 index 0000000000..04a6b418dc --- /dev/null +++ b/conf/fairscheduler.xml.template @@ -0,0 +1,15 @@ + + + + 2 + 1 + FAIR + + + 3 + 2 + FIFO + + + + diff --git a/core/src/test/resources/fairscheduler.xml b/core/src/test/resources/fairscheduler.xml index 5a688b0ebb..6e573b1883 100644 --- a/core/src/test/resources/fairscheduler.xml +++ b/core/src/test/resources/fairscheduler.xml @@ -1,3 +1,4 @@ + 2 From c0c3155c3c0e37ba31cbeeccb21654bdf5ec0cfc Mon Sep 17 00:00:00 2001 From: Lian Cheng Date: Fri, 5 Jul 2013 00:54:10 +0800 Subject: [PATCH 094/419] Bug fix: SPARK-789 https://spark-project.atlassian.net/browse/SPARK-789 --- core/src/main/scala/spark/deploy/master/Master.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3e965e82ac..87f304b6cd 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -278,7 +278,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act exec.state = ExecutorState.KILLED } app.markFinished(state) - app.driver ! ApplicationRemoved(state.toString) + if (state != ApplicationState.FINISHED) { + app.driver ! ApplicationRemoved(state.toString) + } schedule() } } From 7687ed529217161ea681939870f6e1e042139e65 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Thu, 4 Jul 2013 13:48:33 -0700 Subject: [PATCH 095/419] Use standard ASF published avro module instead of a proprietory built one --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 48e623fa1c..5b4c1d48f7 100644 --- a/pom.xml +++ b/pom.xml @@ -620,12 +620,12 @@ org.apache.avro avro - 1.7.1.cloudera.2 + 1.7.1 org.apache.avro avro-ipc - 1.7.1.cloudera.2 + 1.7.1 From e4ff544a8d43a23ad2110556102bb83e34ae71b4 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 5 Jul 2013 10:34:45 +0800 Subject: [PATCH 096/419] Clean StageToInfos periodically when spark.cleaner.ttl is enabled --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f7d60be5db..51b10ed045 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -312,7 +312,7 @@ class DAGScheduler( handleExecutorLost(execId) case completion: CompletionEvent => - sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion.task, + sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion.task, completion.reason, completion.taskInfo, completion.taskMetrics))) handleTaskCompletion(completion) @@ -651,7 +651,7 @@ class DAGScheduler( "(generation " + currentGeneration + ")") } } - + private def handleExecutorGained(execId: String, hostPort: String) { // remove from failedGeneration(execId) ? if (failedGeneration.contains(execId)) { @@ -747,6 +747,10 @@ class DAGScheduler( sizeBefore = pendingTasks.size pendingTasks.clearOldValues(cleanupTime) logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) + + sizeBefore = stageToInfos.size + stageToInfos.clearOldValues(cleanupTime) + logInfo("stageToInfos " + sizeBefore + " --> " + stageToInfos.size) } def stop() { From 729e463f649332c5480d2d175d42d4ba0dd3cb74 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:13:41 -0700 Subject: [PATCH 097/419] Import RidgeRegression example Conflicts: run --- .../main/scala/spark/ml/RidgeRegression.scala | 110 ++++++++++++++++++ .../spark/ml/RidgeRegressionGenerator.scala | 70 +++++++++++ project/SparkBuild.scala | 12 +- 3 files changed, 191 insertions(+), 1 deletion(-) create mode 100644 ml/src/main/scala/spark/ml/RidgeRegression.scala create mode 100644 ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala new file mode 100644 index 0000000000..7896873d44 --- /dev/null +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -0,0 +1,110 @@ +package spark.ml + +import spark._ +import spark.SparkContext._ + +import org.apache.commons.math3.distribution.NormalDistribution +import org.jblas.DoubleMatrix +import org.jblas.Solve + +/** + * Ridge Regression from Joseph Gonzalez's implementation in MLBase + */ + +class RidgeRegressionModel( + val wOpt: DoubleMatrix, + val lambdaOpt: Double, + val lambdas: List[(Double, Double, DoubleMatrix)]) { + + def predict(test_data: spark.RDD[Array[Double]]) = { + test_data.map(x => new DoubleMatrix(1, x.length, x:_*).mmul(this.wOpt)) + } +} + +object RidgeRegression extends Logging { + + def train(data: spark.RDD[(Double, Array[Double])], + lambdaLow: Double = 0.0, + lambdaHigh: Double = 10000.0) = { + + data.cache() + val nfeatures = data.take(1)(0)._2.length + val nexamples = data.count + + // Compute XtX - Size of XtX is nfeatures by nfeatures + val XtX = data.map { + case (y, features) => + val x = new DoubleMatrix(1, features.length, features:_*) + x.transpose().mmul(x) + }.reduce(_.add(_)) + + // Compute Xt*y - Size of Xty is nfeatures by 1 + val Xty = data.map { + case (y, features) => + new DoubleMatrix(features.length, 1, features:_*).mul(y) + }.reduce(_.add(_)) + + // Define a function to compute the leave one out cross validation error + // for a single example + def crossValidate(lambda: Double) = { + // Compute the MLE ridge regression parameter value + + // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty + val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) + val w = Solve.solveSymmetric(XtXlambda, Xty) + + val invXtX = Solve.solveSymmetric(XtXlambda, + DoubleMatrix.eye(nfeatures)) + + // compute the leave one out cross validation score + val cvError = data.map { + case (y, features) => + val x = new DoubleMatrix(features.length, 1, features:_*) + val yhat = w.transpose().mmul(x).get(0) + val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) + val residual = (y - yhat) / (1.0 - H_ii) + residual * residual + }.reduce(_ + _) + (lambda, cvError, w) + } + + // Binary search for the best assignment to lambda. + def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { + val mid = (high - low) / 2 + low + val lowValue = crossValidate((mid - low) / 2 + low) + val highValue = crossValidate((high - mid) / 2 + mid) + val (newLow, newHigh) = if (lowValue._2 < highValue._2) { + (low, mid + (high-low)/4) + } else { + (mid - (high-low)/4, high) + } + if (newHigh - newLow > 1.0E-7) { + lowValue :: highValue :: binSearch(newLow, newHigh) + } else { + List(lowValue, highValue) + } + } + + // Actually compute the best lambda + val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) + + // Find the best parameter set + val (lambdaOpt, cverror, wOpt) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) + + logInfo("RidgeRegression: optimal lambda " + lambdaOpt) + + // Return the model which contains the solution + new RidgeRegressionModel(wOpt, lambdaOpt, lambdas) + } + + def main(args: Array[String]) { + if (args.length != 2) { + println("Usage: RidgeRegression ") + System.exit(1) + } + val sc = new SparkContext(args(0), "RidgeRegression") + val data = RidgeRegressionGenerator.loadData(sc, args(1)) + val model = train(data, 0, 100) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala new file mode 100644 index 0000000000..22a1e4613b --- /dev/null +++ b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala @@ -0,0 +1,70 @@ +package spark.ml + +import spark._ +import spark.SparkContext._ + +import org.apache.commons.math3.distribution.NormalDistribution +import org.jblas.DoubleMatrix + +object RidgeRegressionGenerator { + + // Helper methods to load and save data used for RidgeRegression + // Data format: + // , ... + // where , are feature values in Double and + // is the corresponding label as Double + def loadData(sc: SparkContext, dir: String) = { + val data = sc.textFile(dir).map{ line => + val parts = line.split(",") + val label = parts(0).toDouble + val features = parts(1).trim().split(" ").map(_.toDouble) + (label, features) + } + data + } + + def saveData(data: RDD[(Double, Array[Double])], dir: String) { + val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) + dataStr.saveAsTextFile(dir) + } + + def main(args: Array[String]) { + if (args.length != 2) { + println("Usage: RidgeRegressionGenerator ") + System.exit(1) + } + org.jblas.util.Random.seed(42) + val sc = new SparkContext(args(0), "RidgeRegressionGenerator") + + val nexamples = 1000 + val nfeatures = 100 + val eps = 10 + val parts = 2 + + // Random values distributed uniformly in [-0.5, 0.5] + val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) + w.put(0, 0, 10) + w.put(1, 0, 10) + + val data = sc.parallelize(0 until parts, parts).flatMap { p => + org.jblas.util.Random.seed(42 + p) + val examplesInPartition = nexamples / parts + + val X = DoubleMatrix.rand(examplesInPartition, nfeatures) + val y = X.mmul(w) + + val rnd = new NormalDistribution(0, eps) + rnd.reseedRandomGenerator(42 + p) + + val normalValues = (0 until examplesInPartition).map(_ => rnd.sample()) + val yObs = new DoubleMatrix(examplesInPartition, 1, normalValues:_*).addi(y) + + (0 until examplesInPartition).map(i => + (yObs.get(i, 0), X.getRow(i).toArray) + ) + } + + saveData(data, args(1)) + System.exit(0) + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5e4692162e..731671c23b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -25,7 +25,7 @@ object SparkBuild extends Build { //val HADOOP_MAJOR_VERSION = "2" //val HADOOP_YARN = true - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, ml) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -37,6 +37,8 @@ object SparkBuild extends Build { lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) + lazy val ml = Project("ml", file("ml"), settings = mlSettings) dependsOn (core) + // A configuration to set an alternative publishLocalConfiguration lazy val MavenCompile = config("m2r") extend(Compile) lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") @@ -219,6 +221,14 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") + def mlSettings = examplesSettings ++ Seq( + name := "spark-ml", + libraryDependencies ++= Seq( + "org.jblas" % "jblas" % "1.2.3", + "org.apache.commons" % "commons-math3" % "3.2" + ) + ) + def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", resolvers ++= Seq( From 6a9a9a364ce3b158c4162e401f90eb4d305104e8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 4 Jun 2013 16:27:02 -0700 Subject: [PATCH 098/419] Minor clean up of the RidgeRegression code. I am not even sure why I did this :s. --- .../main/scala/spark/ml/RidgeRegression.scala | 38 +++++++++---------- .../spark/ml/RidgeRegressionGenerator.scala | 38 ++++++++++--------- project/SparkBuild.scala | 2 +- 3 files changed, 39 insertions(+), 39 deletions(-) diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 7896873d44..b8b632e111 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -1,9 +1,7 @@ package spark.ml -import spark._ -import spark.SparkContext._ +import spark.{Logging, RDD, SparkContext} -import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix import org.jblas.Solve @@ -23,39 +21,36 @@ class RidgeRegressionModel( object RidgeRegression extends Logging { - def train(data: spark.RDD[(Double, Array[Double])], - lambdaLow: Double = 0.0, + def train(data: RDD[(Double, Array[Double])], + lambdaLow: Double = 0.0, lambdaHigh: Double = 10000.0) = { data.cache() - val nfeatures = data.take(1)(0)._2.length - val nexamples = data.count + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() // Compute XtX - Size of XtX is nfeatures by nfeatures - val XtX = data.map { - case (y, features) => - val x = new DoubleMatrix(1, features.length, features:_*) - x.transpose().mmul(x) + val XtX: DoubleMatrix = data.map { case (y, features) => + val x = new DoubleMatrix(1, features.length, features:_*) + x.transpose().mmul(x) }.reduce(_.add(_)) // Compute Xt*y - Size of Xty is nfeatures by 1 - val Xty = data.map { - case (y, features) => - new DoubleMatrix(features.length, 1, features:_*).mul(y) + val Xty: DoubleMatrix = data.map { case (y, features) => + new DoubleMatrix(features.length, 1, features:_*).mul(y) }.reduce(_.add(_)) // Define a function to compute the leave one out cross validation error // for a single example - def crossValidate(lambda: Double) = { - // Compute the MLE ridge regression parameter value + def crossValidate(lambda: Double): (Double, Double, DoubleMatrix) = { + // Compute the MLE ridge regression parameter value // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) val w = Solve.solveSymmetric(XtXlambda, Xty) - val invXtX = Solve.solveSymmetric(XtXlambda, - DoubleMatrix.eye(nfeatures)) - + val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) + // compute the leave one out cross validation score val cvError = data.map { case (y, features) => @@ -74,11 +69,12 @@ object RidgeRegression extends Logging { val lowValue = crossValidate((mid - low) / 2 + low) val highValue = crossValidate((high - mid) / 2 + mid) val (newLow, newHigh) = if (lowValue._2 < highValue._2) { - (low, mid + (high-low)/4) + (low, mid + (high-low)/4) } else { (mid - (high-low)/4, high) } if (newHigh - newLow > 1.0E-7) { + // :: is list prepend in Scala. lowValue :: highValue :: binSearch(newLow, newHigh) } else { List(lowValue, highValue) @@ -88,7 +84,7 @@ object RidgeRegression extends Logging { // Actually compute the best lambda val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) - // Find the best parameter set + // Find the best parameter set by taking the lowest cverror. val (lambdaOpt, cverror, wOpt) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) logInfo("RidgeRegression: optimal lambda " + lambdaOpt) diff --git a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala index 22a1e4613b..ff8640bb50 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala @@ -1,11 +1,11 @@ package spark.ml -import spark._ -import spark.SparkContext._ +import spark.{RDD, SparkContext} import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix + object RidgeRegressionGenerator { // Helper methods to load and save data used for RidgeRegression @@ -23,30 +23,34 @@ object RidgeRegressionGenerator { data } - def saveData(data: RDD[(Double, Array[Double])], dir: String) { + private def saveData(data: RDD[(Double, Array[Double])], dir: String) { val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) dataStr.saveAsTextFile(dir) } def main(args: Array[String]) { if (args.length != 2) { - println("Usage: RidgeRegressionGenerator ") + println("Usage: RidgeRegressionGenerator " + + " ") System.exit(1) } - org.jblas.util.Random.seed(42) - val sc = new SparkContext(args(0), "RidgeRegressionGenerator") - val nexamples = 1000 - val nfeatures = 100 + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 100 + val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 10 - val parts = 2 + + org.jblas.util.Random.seed(42) + val sc = new SparkContext(sparkMaster, "RidgeRegressionGenerator") // Random values distributed uniformly in [-0.5, 0.5] val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) w.put(0, 0, 10) w.put(1, 0, 10) - val data = sc.parallelize(0 until parts, parts).flatMap { p => + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until parts, parts).flatMap { p => org.jblas.util.Random.seed(42 + p) val examplesInPartition = nexamples / parts @@ -56,15 +60,15 @@ object RidgeRegressionGenerator { val rnd = new NormalDistribution(0, eps) rnd.reseedRandomGenerator(42 + p) - val normalValues = (0 until examplesInPartition).map(_ => rnd.sample()) - val yObs = new DoubleMatrix(examplesInPartition, 1, normalValues:_*).addi(y) - - (0 until examplesInPartition).map(i => + val normalValues = Array.fill[Double](examplesInPartition)(rnd.sample()) + val yObs = new DoubleMatrix(normalValues).addi(y) + + Iterator.tabulate(examplesInPartition) { i => (yObs.get(i, 0), X.getRow(i).toArray) - ) + } } - saveData(data, args(1)) - System.exit(0) + saveData(data, outputPath) + sc.stop() } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 731671c23b..aa877ad4a7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -221,7 +221,7 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") - def mlSettings = examplesSettings ++ Seq( + def mlSettings = sharedSettings ++ Seq( name := "spark-ml", libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3", From c070decb8e746bb295b3d95f49713d0d389f92b3 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 6 Jun 2013 11:04:33 -0700 Subject: [PATCH 099/419] Add methods to normalize the data before training Also update model after training based appropriately. --- .../main/scala/spark/ml/RidgeRegression.scala | 124 ++++++++++++++++-- 1 file changed, 111 insertions(+), 13 deletions(-) diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index b8b632e111..881316c0fc 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -1,6 +1,7 @@ package spark.ml import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ import org.jblas.DoubleMatrix import org.jblas.Solve @@ -11,20 +12,32 @@ import org.jblas.Solve class RidgeRegressionModel( val wOpt: DoubleMatrix, + val bOpt: Double, val lambdaOpt: Double, val lambdas: List[(Double, Double, DoubleMatrix)]) { def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map(x => new DoubleMatrix(1, x.length, x:_*).mmul(this.wOpt)) + test_data.map(x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.wOpt)).get(0) + this.bOpt) } } +case class RidgeRegressionData( + val data: RDD[(Double, Array[Double])], + val normalizedData: RDD[(Double, Array[Double])], + val yMean: Double, + val xColMean: Array[Double], + val xColSd: Array[Double] +) + object RidgeRegression extends Logging { - def train(data: RDD[(Double, Array[Double])], + def train(inputData: RDD[(Double, Array[Double])], lambdaLow: Double = 0.0, lambdaHigh: Double = 10000.0) = { + val ridgeData = normalize(inputData) + val data = ridgeData.normalizedData + data.cache() val nfeatures: Int = data.take(1)(0)._2.length val nexamples: Long = data.count() @@ -51,19 +64,27 @@ object RidgeRegression extends Logging { val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) - // compute the leave one out cross validation score - val cvError = data.map { + // compute the generalized cross validation score + // TODO: Is there a way to calculate this using one pass on the data ? + val H_ii_mean = data.map { + case (y, features) => + val x = new DoubleMatrix(features.length, 1, features:_*) + val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) + H_ii + }.reduce(_ + _) / nexamples + + val gcv = data.map { case (y, features) => val x = new DoubleMatrix(features.length, 1, features:_*) val yhat = w.transpose().mmul(x).get(0) - val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) - val residual = (y - yhat) / (1.0 - H_ii) + val residual = (y - yhat) / (1.0 - H_ii_mean) residual * residual - }.reduce(_ + _) - (lambda, cvError, w) + }.reduce(_ + _) / nexamples + + (lambda, gcv, w) } - // Binary search for the best assignment to lambda. + // Binary search for the best assignment to lambda. def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { val mid = (high - low) / 2 + low val lowValue = crossValidate((mid - low) / 2 + low) @@ -87,12 +108,89 @@ object RidgeRegression extends Logging { // Find the best parameter set by taking the lowest cverror. val (lambdaOpt, cverror, wOpt) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) - logInfo("RidgeRegression: optimal lambda " + lambdaOpt) + // Return the model which contains the solution + val trainModel = new RidgeRegressionModel(wOpt, 0.0, lambdaOpt, lambdas) + val normModel = normalizeModel(trainModel, ridgeData.xColSd, ridgeData.xColMean, ridgeData.yMean) - // Return the model which contains the solution - new RidgeRegressionModel(wOpt, lambdaOpt, lambdas) + logInfo("RidgeRegression: optimal lambda " + normModel.lambdaOpt) + logInfo("RidgeRegression: optimal weights " + normModel.wOpt) + logInfo("RidgeRegression: optimal intercept " + normModel.bOpt) + logInfo("RidgeRegression: optimal GCV " + cverror) + + normModel } + /** + * yMu = Mean[Y] + * xMuVec = Mean[X] + * xSigmaVec = StdDev[X] + * + * // Shift the data + * Xtrain = (X - xMuVec) / xSigmaVec + * Ytrain = Y - yMu + */ + def normalize(data: RDD[(Double, Array[Double])]) = { + data.cache() + + val nexamples: Long = data.count() + val nfeatures: Int = data.take(1)(0)._2.length + + // Calculate the mean for Y + val yMean: Double = data.map { case (y, features) => y }.reduce(_ + _) / nexamples + + // NOTE: We shuffle X by column here to compute column sum and sum of squares. + val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + (0 until nCols).map(i => (i, (features(i), features(i)*features(i)))) + }.reduceByKey { case(x1, x2) => + (x1._1 + x2._1, x1._2 + x2._2) + } + val xColSumsMap = xColSumSq.collectAsMap() + + // Compute mean and unbiased variance using column sums + val xColMeans = (0 until nfeatures).map(x => xColSumsMap(x)._1 / nexamples).toArray + val xColSd = (0 until nfeatures).map {x => + val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) + math.sqrt(v) + }.toArray + + // Shift the data + val normalizedData = data.map { case(y, features) => + val yNormalized = y - yMean + val featuresNormalized = (0 until nfeatures).map( + column => (features(column) - xColMeans(column)) / xColSd(column) + ).toArray + (yNormalized, featuresNormalized) + } + new RidgeRegressionData(data, normalizedData, yMean, xColMeans, xColSd) + } + + /** + * Augment and return then final model (derivation): + * y = w' ( (xPred - xMu) / xSigma ) + yMu + * y = w' xPred/sigma + (yMu - w' (xMu/ xSigmaVec) + * Note that the / operator is point wise divions + * + * model.w = w' / sigma // point wise division + * model.b = yMu - w' * (xMu / xSigmaVec) // scalar offset + * + * // Make predictions + * yPred = model.w' * xPred + model.b + */ + def normalizeModel(model: RidgeRegressionModel, + xColSd: Array[Double], xColMeans: Array[Double], + yMean: Double) = { + val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) + val colMeanMat = new DoubleMatrix(xColMeans.length, 1, xColMeans:_*) + + val wOpt = model.wOpt.div(colSdMat) + val bOpt = yMean - model.wOpt.transpose().mmul(colMeanMat.div(colSdMat)).get(0) + + new RidgeRegressionModel(wOpt, bOpt, model.lambdaOpt, model.lambdas) + } + + def main(args: Array[String]) { if (args.length != 2) { println("Usage: RidgeRegression ") @@ -100,7 +198,7 @@ object RidgeRegression extends Logging { } val sc = new SparkContext(args(0), "RidgeRegression") val data = RidgeRegressionGenerator.loadData(sc, args(1)) - val model = train(data, 0, 100) + val model = train(data, 0, 1000) sc.stop() } } From c8169c0a3331c8c89cd963f8f14ea69dbe3ef966 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 6 Jun 2013 11:05:11 -0700 Subject: [PATCH 100/419] Add LPSA data set. Data from http://www-stat.stanford.edu/~tibs/ElemStatLearn/datasets/prostate.data --- ml/data/ridge-data/lpsa.data | 67 ++++++++++++++++++++++++++++++++++++ 1 file changed, 67 insertions(+) create mode 100644 ml/data/ridge-data/lpsa.data diff --git a/ml/data/ridge-data/lpsa.data b/ml/data/ridge-data/lpsa.data new file mode 100644 index 0000000000..fdd16e36b4 --- /dev/null +++ b/ml/data/ridge-data/lpsa.data @@ -0,0 +1,67 @@ +-0.4307829,-1.63735562648104 -2.00621178480549 -1.86242597251066 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +-0.1625189,-1.98898046126935 -0.722008756122123 -0.787896192088153 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +-0.1625189,-1.57881887548545 -2.1887840293994 1.36116336875686 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.155348103855541 +-0.1625189,-2.16691708463163 -0.807993896938655 -0.787896192088153 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +0.3715636,-0.507874475300631 -0.458834049396776 -0.250631301876899 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +0.7654678,-2.03612849966376 -0.933954647105133 -1.86242597251066 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +0.8544153,-0.557312518810673 -0.208756571683607 -0.787896192088153 0.990146852537193 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.2669476,-0.929360463147704 -0.0578991819441687 0.152317365781542 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.2669476,-2.28833047634983 -0.0706369432557794 -0.116315079324086 0.80409888772376 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.2669476,0.223498042876113 -1.41471935455355 -0.116315079324086 -1.02470580167082 -0.522940888712441 -0.29928234305568 0.342627053981254 0.199211097885341 +1.3480731,0.107785900236813 -1.47221551299731 0.420949810887169 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.687186906466865 +1.446919,0.162180092313795 -1.32557369901905 0.286633588334355 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.4701758,-1.49795329918548 -0.263601072284232 0.823898478545609 0.788388310173035 -0.522940888712441 -0.29928234305568 0.342627053981254 0.199211097885341 +1.4929041,0.796247055396743 0.0476559407005752 0.286633588334355 -1.02470580167082 -0.522940888712441 0.394013435896129 -1.04215728919298 -0.864466507337306 +1.5581446,-1.62233848461465 -0.843294091975396 -3.07127197548598 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.5993876,-0.990720665490831 0.458513517212311 0.823898478545609 1.07379746308195 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.6389967,-0.171901281967138 -0.489197399065355 -0.65357996953534 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.6956156,-1.60758252338831 -0.590700340358265 -0.65357996953534 -0.619561070667254 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.7137979,0.366273918511144 -0.414014962912583 -0.116315079324086 0.232904453212813 -0.522940888712441 0.971228997418125 0.342627053981254 1.26288870310799 +1.8000583,-0.710307384579833 0.211731938156277 0.152317365781542 -1.02470580167082 -0.522940888712441 -0.442797990776478 0.342627053981254 1.61744790484887 +1.8484548,-0.262791728113881 -1.16708345615721 0.420949810887169 0.0846342590816532 -0.522940888712441 0.163172393491611 0.342627053981254 1.97200710658975 +1.8946169,0.899043117369237 -0.590700340358265 0.152317365781542 -1.02470580167082 -0.522940888712441 1.28643254437683 -1.04215728919298 -0.864466507337306 +1.9242487,-0.903451690500615 1.07659722048274 0.152317365781542 1.28380453408541 -0.522940888712441 -0.442797990776478 -1.04215728919298 -0.864466507337306 +2.008214,-0.0633337899773081 -1.38088970920094 0.958214701098423 0.80409888772376 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.0476928,-1.15393789990757 -0.961853075398404 -0.116315079324086 -1.02470580167082 -0.522940888712441 -0.442797990776478 -1.04215728919298 -0.864466507337306 +2.1575593,0.0620203721138446 0.0657973885499142 1.22684714620405 -0.468824786336838 -0.522940888712441 1.31421001659859 1.72741139715549 -0.332627704725983 +2.1916535,-0.75731027755674 -2.92717970468456 0.018001143228728 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 +2.2137539,1.11226993252773 1.06484916245061 0.555266033439982 0.877691038550889 1.89254797819741 1.43890404648442 0.342627053981254 0.376490698755783 +2.2772673,-0.468768642850639 -1.43754788774533 -1.05652863719378 0.576050411655607 -0.522940888712441 0.0120483832567209 0.342627053981254 -0.687186906466865 +2.2975726,-0.618884859896728 -1.1366360750781 -0.519263746982526 -1.02470580167082 -0.522940888712441 -0.863171185425945 3.11219574032972 1.97200710658975 +2.3272777,-0.651431999123483 0.55329161145762 -0.250631301876899 1.11210019001038 -0.522940888712441 -0.179808625688859 -1.04215728919298 -0.864466507337306 +2.5217206,0.115499102435224 -0.512233676577595 0.286633588334355 1.13650173283446 -0.522940888712441 -0.179808625688859 0.342627053981254 -0.155348103855541 +2.5533438,0.266341329949937 -0.551137885443386 -0.384947524429713 0.354857790686005 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 +2.5687881,1.16902610257751 0.855491905752846 2.03274448152093 1.22628985326088 1.89254797819741 2.02833774827712 3.11219574032972 2.68112551007152 +2.6567569,-0.218972367124187 0.851192298581141 0.555266033439982 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 0.908329501367106 +2.677591,0.263121415733908 1.4142681068416 0.018001143228728 1.35980653053822 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.7180005,-0.0704736333296423 1.52000996595417 0.286633588334355 1.39364261119802 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 +2.7942279,-0.751957286017338 0.316843561689933 -1.99674219506348 0.911736065044475 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.8063861,-0.685277652430997 1.28214038482516 0.823898478545609 0.232904453212813 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.155348103855541 +2.8124102,-0.244991501432929 0.51882005949686 -0.384947524429713 0.823246560137838 -0.522940888712441 -0.863171185425945 0.342627053981254 0.553770299626224 +2.8419982,-0.75731027755674 2.09041984898851 1.22684714620405 1.53428167116843 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.8535925,1.20962937075363 -0.242882661178889 1.09253092365124 -1.02470580167082 -0.522940888712441 1.24263233939889 3.11219574032972 2.50384590920108 +2.9204698,0.570886990493502 0.58243883987948 0.555266033439982 1.16006887775962 -0.522940888712441 1.07357183940747 0.342627053981254 1.61744790484887 +2.9626924,0.719758684343624 0.984970304132004 1.09253092365124 1.52137230773457 -0.522940888712441 -0.179808625688859 0.342627053981254 -0.509907305596424 +2.9626924,-1.52406140158064 1.81975700990333 0.689582255992796 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.9729753,-0.132431544081234 2.68769877553723 1.09253092365124 1.53428167116843 -0.522940888712441 -0.442797990776478 0.342627053981254 -0.687186906466865 +3.0130809,0.436161292804989 -0.0834447307428255 -0.519263746982526 -1.02470580167082 1.89254797819741 1.07357183940747 0.342627053981254 1.26288870310799 +3.0373539,-0.161195191984091 -0.671900359186746 1.7641120364153 1.13650173283446 -0.522940888712441 -0.863171185425945 0.342627053981254 0.0219314970149 +3.2752562,1.39927182372944 0.513852869452676 0.689582255992796 -1.02470580167082 1.89254797819741 1.49394503405693 0.342627053981254 -0.155348103855541 +3.3375474,1.51967002306341 -0.852203755696565 0.555266033439982 -0.104527297798983 1.89254797819741 1.85927724828569 0.342627053981254 0.908329501367106 +3.3928291,0.560725834706224 1.87867703391426 1.09253092365124 1.39364261119802 -0.522940888712441 0.486423065822545 0.342627053981254 1.26288870310799 +3.4355988,1.00765532502814 1.69426310090641 1.89842825896812 1.53428167116843 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.509907305596424 +3.4578927,1.10152996153577 -0.10927271844907 0.689582255992796 -1.02470580167082 1.89254797819741 1.97630171771485 0.342627053981254 1.61744790484887 +3.5160131,0.100001934217311 -1.30380956369388 0.286633588334355 0.316555063757567 -0.522940888712441 0.28786643052924 0.342627053981254 0.553770299626224 +3.5307626,0.987291634724086 -0.36279314978779 -0.922212414640967 0.232904453212813 -0.522940888712441 1.79270085261407 0.342627053981254 1.26288870310799 +3.5652984,1.07158528137575 0.606453149641961 1.7641120364153 -0.432854616994416 1.89254797819741 0.528504607720369 0.342627053981254 0.199211097885341 +3.5876769,0.180156323255198 0.188987436375017 -0.519263746982526 1.09956763075594 -0.522940888712441 0.708239632330506 0.342627053981254 0.199211097885341 +3.6309855,1.65687973755377 -0.256675483533719 0.018001143228728 -1.02470580167082 1.89254797819741 1.79270085261407 0.342627053981254 1.26288870310799 +3.6800909,0.5720085322365 0.239854450210939 -0.787896192088153 1.0605418233138 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +3.7123518,0.323806133438225 -0.606717660886078 -0.250631301876899 -1.02470580167082 1.89254797819741 0.342907418101747 0.342627053981254 0.199211097885341 +3.9843437,1.23668206715898 2.54220539083611 0.152317365781542 -1.02470580167082 1.89254797819741 1.89037692416194 0.342627053981254 1.26288870310799 +3.993603,0.180156323255198 0.154448192444669 1.62979581386249 0.576050411655607 1.89254797819741 0.708239632330506 0.342627053981254 1.79472750571931 +4.029806,1.60906277046565 1.10378605019827 0.555266033439982 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +4.1295508,1.0036214996026 0.113496885050331 -0.384947524429713 0.860016436332751 1.89254797819741 -0.863171185425945 0.342627053981254 -0.332627704725983 +4.3851468,1.25591974271076 0.577607033774471 0.555266033439982 -1.02470580167082 1.89254797819741 1.07357183940747 0.342627053981254 1.26288870310799 +4.6844434,2.09650591351268 0.625488598331018 -2.66832330782754 -1.02470580167082 1.89254797819741 1.67954222367555 0.342627053981254 0.553770299626224 +5.477509,1.30028987435881 0.338383613253713 0.555266033439982 1.00481276295349 1.89254797819741 1.24263233939889 0.342627053981254 1.97200710658975 From 4dc13bf5be713443f22ca9c27530fb1d3cf48993 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 7 Jun 2013 15:10:13 -0700 Subject: [PATCH 101/419] Revert back to closed form CV error --- .../main/scala/spark/ml/RidgeRegression.scala | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 881316c0fc..2288a284e2 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -65,23 +65,16 @@ object RidgeRegression extends Logging { val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) // compute the generalized cross validation score - // TODO: Is there a way to calculate this using one pass on the data ? - val H_ii_mean = data.map { - case (y, features) => - val x = new DoubleMatrix(features.length, 1, features:_*) - val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) - H_ii - }.reduce(_ + _) / nexamples - - val gcv = data.map { + val cverror = data.map { case (y, features) => val x = new DoubleMatrix(features.length, 1, features:_*) val yhat = w.transpose().mmul(x).get(0) - val residual = (y - yhat) / (1.0 - H_ii_mean) + val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) + val residual = (y - yhat) / (1.0 - H_ii) residual * residual }.reduce(_ + _) / nexamples - (lambda, gcv, w) + (lambda, cverror, w) } // Binary search for the best assignment to lambda. @@ -115,7 +108,7 @@ object RidgeRegression extends Logging { logInfo("RidgeRegression: optimal lambda " + normModel.lambdaOpt) logInfo("RidgeRegression: optimal weights " + normModel.wOpt) logInfo("RidgeRegression: optimal intercept " + normModel.bOpt) - logInfo("RidgeRegression: optimal GCV " + cverror) + logInfo("RidgeRegression: cross-validation error " + cverror) normModel } From b9d9b6f98168ae211e36e94ee4c2d8defa525169 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 8 Jun 2013 17:04:30 -0700 Subject: [PATCH 102/419] Add a unit test for Ridge Regression --- ml/src/test/resources/log4j.properties | 11 +++++ .../scala/spark/ml/RidgeRegressionSuite.scala | 41 +++++++++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 ml/src/test/resources/log4j.properties create mode 100644 ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala diff --git a/ml/src/test/resources/log4j.properties b/ml/src/test/resources/log4j.properties new file mode 100644 index 0000000000..390c92763c --- /dev/null +++ b/ml/src/test/resources/log4j.properties @@ -0,0 +1,11 @@ +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=ml/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala new file mode 100644 index 0000000000..cc46602edd --- /dev/null +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -0,0 +1,41 @@ +package spark.ml + +import spark.SparkContext +import spark.SparkContext._ + +import org.apache.commons.math3.distribution.NormalDistribution +import org.scalatest.FunSuite + +class RidgeRegressionSuite extends FunSuite { + + // Test if we can correctly learn Y = 3 + X1 + X2 when + // X1 and X2 are collinear. + test("multi-collinear variables") { + val rnd = new NormalDistribution(0, 1) + rnd.reseedRandomGenerator(43) + val sc = new SparkContext("local", "test") + val x1 = Array.fill[Double](20)(rnd.sample()) + + // Pick a mean close to mean of x1 + val rnd1 = new NormalDistribution(0.1, 0.01) + rnd1.reseedRandomGenerator(42) + val x2 = Array.fill[Double](20)(rnd1.sample()) + + val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray + + val y = xMat.map(i => 3 + i(0) + i(1)) + val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray + + val testRDD = sc.parallelize(testData, 2) + val model = RidgeRegression.train(testRDD, 0, 10) + + assert(model.bOpt >= 2.9 && model.bOpt <= 3.1) + assert(model.wOpt.length === 2) + assert(model.wOpt.get(0) >= 0.9 && model.wOpt.get(0) <= 1.1) + assert(model.wOpt.get(1) >= 0.9 && model.wOpt.get(1) <= 1.1) + + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } +} From 282c8ed78843fc9a3555b4b6dbdb709d2632d360 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 10 Jun 2013 23:16:11 -0700 Subject: [PATCH 103/419] Add LogisticRegression using StochasticGradientDescent. Also refactor RidgeRegression and LogisticRegression to re-use code and update the test as well --- ml/src/main/scala/spark/ml/Gradient.scala | 24 ++++ .../main/scala/spark/ml/GradientDescent.scala | 46 ++++++ .../scala/spark/ml/LogisticRegression.scala | 77 ++++++++++ .../ml/LogisticRegressionGenerator.scala | 40 ++++++ ml/src/main/scala/spark/ml/MLUtils.scala | 27 ++++ ml/src/main/scala/spark/ml/Regression.scala | 53 +++++++ .../main/scala/spark/ml/RidgeRegression.scala | 134 +++++------------- .../spark/ml/RidgeRegressionGenerator.scala | 24 +--- ml/src/main/scala/spark/ml/Updater.scala | 16 +++ .../scala/spark/ml/RidgeRegressionSuite.scala | 8 +- 10 files changed, 327 insertions(+), 122 deletions(-) create mode 100644 ml/src/main/scala/spark/ml/Gradient.scala create mode 100644 ml/src/main/scala/spark/ml/GradientDescent.scala create mode 100644 ml/src/main/scala/spark/ml/LogisticRegression.scala create mode 100644 ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala create mode 100644 ml/src/main/scala/spark/ml/MLUtils.scala create mode 100644 ml/src/main/scala/spark/ml/Regression.scala create mode 100644 ml/src/main/scala/spark/ml/Updater.scala diff --git a/ml/src/main/scala/spark/ml/Gradient.scala b/ml/src/main/scala/spark/ml/Gradient.scala new file mode 100644 index 0000000000..a7e8327133 --- /dev/null +++ b/ml/src/main/scala/spark/ml/Gradient.scala @@ -0,0 +1,24 @@ +package spark.ml + +import org.jblas.DoubleMatrix + +abstract class Gradient extends Serializable { + def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) +} + +class LogisticGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + val margin: Double = -1.0 * data.dot(weights) + val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label + + val gradient = data.mul(gradientMultiplier) + val loss = if (margin > 0) { + math.log(1 + math.exp(0 - margin)) + } else { + math.log(1 + math.exp(margin)) - margin + } + (gradient, loss) + } +} diff --git a/ml/src/main/scala/spark/ml/GradientDescent.scala b/ml/src/main/scala/spark/ml/GradientDescent.scala new file mode 100644 index 0000000000..80d4c44a13 --- /dev/null +++ b/ml/src/main/scala/spark/ml/GradientDescent.scala @@ -0,0 +1,46 @@ +package spark.ml + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +import scala.collection.mutable.ArrayBuffer + + +object GradientDescent { + + def runMiniBatchSGD( + data: RDD[(Double, Array[Double])], + gradient: Gradient, + updater: Updater, + stepSize: Double, + numIters: Int, + miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + + val lossHistory = new ArrayBuffer[Double] + + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() + val miniBatchSize = nexamples * miniBatchFraction + + var weights = DoubleMatrix.ones(nfeatures) + var reg_val = 0.0 + + for (i <- 1 to numIters) { + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42).map { + case (y, features) => + val featuresRow = new DoubleMatrix(features.length, 1, features:_*) + val (grad, loss) = gradient.compute(featuresRow, y, weights) + (grad, loss) + }.reduce((a, b) => (a._1.add(b._1), a._2 + b._2)) + + lossHistory.append(lossSum / miniBatchSize + reg_val) + val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) + weights = update._1 + reg_val = update._2 + } + + (weights, lossHistory.toArray) + } +} diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala new file mode 100644 index 0000000000..5647bf5a84 --- /dev/null +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -0,0 +1,77 @@ +package spark.ml + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +/** + * Logistic Regression using Stochastic Gradient Descent. + */ +class LogisticRegressionModel( + weights: DoubleMatrix, + intercept: Double, + val losses: Array[Double]) extends RegressionModel(weights, intercept) { + + override def predict(test_data: spark.RDD[Array[Double]]) = { + test_data.map { x => + val margin = (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + 1.0/(1.0 + math.exp(margin * -1)) + } + } +} + +class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends RegressionData(data) { + override def normalizeData() = { + // Shift only the features for LogisticRegression + data.map { case(y, features) => + val featuresNormalized = (0 until nfeatures).map( + column => (features(column) - xColMean(column)) / xColSd(column) + ).toArray + (y, featuresNormalized) + } + } + + override def scaleModel(m: RegressionModel) = { + val model = m.asInstanceOf[LogisticRegressionModel] + val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) + val colMeanMat = new DoubleMatrix(xColMean.length, 1, xColMean:_*) + + val weights = model.weights.div(colSdMat) + val intercept = -1.0 * model.weights.transpose().mmul(colMeanMat.div(colSdMat)).get(0) + + new LogisticRegressionModel(weights, intercept, model.losses) + } +} + +object LogisticRegression extends Logging { + val STEP_SIZE = 1.0 + val MINI_BATCH_FRACTION = 1.0 + + def train(input: RDD[(Double, Array[Double])], numIters: Int) = { + input.cache() + + val lrData = new LogisticRegressionData(input) + val data = lrData.normalizeData() + val (weights, losses) = GradientDescent.runMiniBatchSGD( + data, new LogisticGradient(), new SimpleUpdater(), STEP_SIZE, numIters, MINI_BATCH_FRACTION) + + val computedModel = new LogisticRegressionModel(weights, 0, losses) + val model = lrData.scaleModel(computedModel) + logInfo("Final model weights " + model.weights) + logInfo("Final model intercept " + model.intercept) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(",")) + model + } + + def main(args: Array[String]) { + if (args.length != 3) { + println("Usage: LogisticRegression ") + System.exit(1) + } + val sc = new SparkContext(args(0), "LogisticRegression") + val data = MLUtils.loadData(sc, args(1)) + val model = train(data, args(2).toInt) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala b/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala new file mode 100644 index 0000000000..ff46cfa46d --- /dev/null +++ b/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala @@ -0,0 +1,40 @@ +package spark.ml + +import spark.{RDD, SparkContext} + +import org.apache.commons.math3.distribution.NormalDistribution +import org.jblas.DoubleMatrix + +object LogisticRegressionGenerator { + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LogisticRegressionGenerator " + + " ") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 + val parts: Int = if (args.length > 4) args(4).toInt else 2 + val eps = 3 + + val sc = new SparkContext(sparkMaster, "LogisticRegressionGenerator") + + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => + val rnd = new NormalDistribution(0, 1) + rnd.reseedRandomGenerator(42 + idx) + + val y = if (idx % 2 == 0) 0 else 1 + val x = Array.fill[Double](nfeatures) { + rnd.sample() + (y * eps) + } + (y, x) + } + + MLUtils.saveData(data, outputPath) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/MLUtils.scala b/ml/src/main/scala/spark/ml/MLUtils.scala new file mode 100644 index 0000000000..eab5db61bd --- /dev/null +++ b/ml/src/main/scala/spark/ml/MLUtils.scala @@ -0,0 +1,27 @@ +package spark.ml + +import spark.{RDD, SparkContext} + +object MLUtils { + + // Helper methods to load and save data + // Data format: + // , ... + // where , are feature values in Double and + // is the corresponding label as Double + def loadData(sc: SparkContext, dir: String) = { + val data = sc.textFile(dir).map{ line => + val parts = line.split(",") + val label = parts(0).toDouble + val features = parts(1).trim().split(" ").map(_.toDouble) + (label, features) + } + data + } + + def saveData(data: RDD[(Double, Array[Double])], dir: String) { + val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) + dataStr.saveAsTextFile(dir) + } + +} diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/Regression.scala new file mode 100644 index 0000000000..e19d58396b --- /dev/null +++ b/ml/src/main/scala/spark/ml/Regression.scala @@ -0,0 +1,53 @@ +package spark.ml + +import java.io._ + +import spark.{RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +abstract class RegressionModel( + val weights: DoubleMatrix, + val intercept: Double) { + + def predict(test_data: RDD[Array[Double]]): RDD[Double] +} + +abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Serializable { + var yMean: Double = 0.0 + var xColMean: Array[Double] = null + var xColSd: Array[Double] = null + var nfeatures: Int = 0 + var nexamples: Long = 0 + + // This will populate yMean, xColMean and xColSd + calculateStats() + + def normalizeData(): RDD[(Double, Array[Double])] + def scaleModel(model: RegressionModel): RegressionModel + + def calculateStats() { + this.nexamples = data.count() + this.nfeatures = data.take(1)(0)._2.length + + this.yMean = data.map { case (y, features) => y }.reduce(_ + _) / nexamples + + // NOTE: We shuffle X by column here to compute column sum and sum of squares. + val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + (0 until nCols).map(i => (i, (features(i), features(i)*features(i)))) + }.reduceByKey { case(x1, x2) => + (x1._1 + x2._1, x1._2 + x2._2) + } + val xColSumsMap = xColSumSq.collectAsMap() + + // Compute mean and unbiased variance using column sums + this.xColMean = (0 until nfeatures).map(x => xColSumsMap(x)._1 / nexamples).toArray + this.xColSd = (0 until nfeatures).map {x => + val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) + math.sqrt(v) + }.toArray + } +} diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 2288a284e2..c030223b85 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -9,25 +9,39 @@ import org.jblas.Solve /** * Ridge Regression from Joseph Gonzalez's implementation in MLBase */ - class RidgeRegressionModel( - val wOpt: DoubleMatrix, - val bOpt: Double, + weights: DoubleMatrix, + intercept: Double, val lambdaOpt: Double, - val lambdas: List[(Double, Double, DoubleMatrix)]) { + val lambdas: List[(Double, Double, DoubleMatrix)]) extends RegressionModel(weights, intercept) { - def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map(x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.wOpt)).get(0) + this.bOpt) + override def predict(test_data: spark.RDD[Array[Double]]) = { + test_data.map(x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept) } } -case class RidgeRegressionData( - val data: RDD[(Double, Array[Double])], - val normalizedData: RDD[(Double, Array[Double])], - val yMean: Double, - val xColMean: Array[Double], - val xColSd: Array[Double] -) +class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends RegressionData(data) { + override def normalizeData() = { + data.map { case(y, features) => + val yNormalized = y - yMean + val featuresNormalized = (0 until nfeatures).map( + column => (features(column) - xColMean(column)) / xColSd(column) + ).toArray + (yNormalized, featuresNormalized) + } + } + + override def scaleModel(m: RegressionModel) = { + val model = m.asInstanceOf[RidgeRegressionModel] + val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) + val colMeanMat = new DoubleMatrix(xColMean.length, 1, xColMean:_*) + + val weights = model.weights.div(colSdMat) + val intercept = yMean - model.weights.transpose().mmul(colMeanMat.div(colSdMat)).get(0) + + new RidgeRegressionModel(weights, intercept, model.lambdaOpt, model.lambdas) + } +} object RidgeRegression extends Logging { @@ -35,12 +49,11 @@ object RidgeRegression extends Logging { lambdaLow: Double = 0.0, lambdaHigh: Double = 10000.0) = { - val ridgeData = normalize(inputData) - val data = ridgeData.normalizedData - - data.cache() - val nfeatures: Int = data.take(1)(0)._2.length - val nexamples: Long = data.count() + inputData.cache() + val ridgeData = new RidgeRegressionData(inputData) + val data = ridgeData.normalizeData() + val nfeatures: Int = ridgeData.nfeatures + val nexamples: Long = ridgeData.nexamples // Compute XtX - Size of XtX is nfeatures by nfeatures val XtX: DoubleMatrix = data.map { case (y, features) => @@ -99,98 +112,27 @@ object RidgeRegression extends Logging { val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) // Find the best parameter set by taking the lowest cverror. - val (lambdaOpt, cverror, wOpt) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) + val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) // Return the model which contains the solution - val trainModel = new RidgeRegressionModel(wOpt, 0.0, lambdaOpt, lambdas) - val normModel = normalizeModel(trainModel, ridgeData.xColSd, ridgeData.xColMean, ridgeData.yMean) + val trainModel = new RidgeRegressionModel(weights, 0.0, lambdaOpt, lambdas) + val normModel = ridgeData.scaleModel(trainModel) logInfo("RidgeRegression: optimal lambda " + normModel.lambdaOpt) - logInfo("RidgeRegression: optimal weights " + normModel.wOpt) - logInfo("RidgeRegression: optimal intercept " + normModel.bOpt) + logInfo("RidgeRegression: optimal weights " + normModel.weights) + logInfo("RidgeRegression: optimal intercept " + normModel.intercept) logInfo("RidgeRegression: cross-validation error " + cverror) normModel } - /** - * yMu = Mean[Y] - * xMuVec = Mean[X] - * xSigmaVec = StdDev[X] - * - * // Shift the data - * Xtrain = (X - xMuVec) / xSigmaVec - * Ytrain = Y - yMu - */ - def normalize(data: RDD[(Double, Array[Double])]) = { - data.cache() - - val nexamples: Long = data.count() - val nfeatures: Int = data.take(1)(0)._2.length - - // Calculate the mean for Y - val yMean: Double = data.map { case (y, features) => y }.reduce(_ + _) / nexamples - - // NOTE: We shuffle X by column here to compute column sum and sum of squares. - val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => - val nCols = features.length - // Traverse over every column and emit (col, value, value^2) - (0 until nCols).map(i => (i, (features(i), features(i)*features(i)))) - }.reduceByKey { case(x1, x2) => - (x1._1 + x2._1, x1._2 + x2._2) - } - val xColSumsMap = xColSumSq.collectAsMap() - - // Compute mean and unbiased variance using column sums - val xColMeans = (0 until nfeatures).map(x => xColSumsMap(x)._1 / nexamples).toArray - val xColSd = (0 until nfeatures).map {x => - val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) - math.sqrt(v) - }.toArray - - // Shift the data - val normalizedData = data.map { case(y, features) => - val yNormalized = y - yMean - val featuresNormalized = (0 until nfeatures).map( - column => (features(column) - xColMeans(column)) / xColSd(column) - ).toArray - (yNormalized, featuresNormalized) - } - new RidgeRegressionData(data, normalizedData, yMean, xColMeans, xColSd) - } - - /** - * Augment and return then final model (derivation): - * y = w' ( (xPred - xMu) / xSigma ) + yMu - * y = w' xPred/sigma + (yMu - w' (xMu/ xSigmaVec) - * Note that the / operator is point wise divions - * - * model.w = w' / sigma // point wise division - * model.b = yMu - w' * (xMu / xSigmaVec) // scalar offset - * - * // Make predictions - * yPred = model.w' * xPred + model.b - */ - def normalizeModel(model: RidgeRegressionModel, - xColSd: Array[Double], xColMeans: Array[Double], - yMean: Double) = { - val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) - val colMeanMat = new DoubleMatrix(xColMeans.length, 1, xColMeans:_*) - - val wOpt = model.wOpt.div(colSdMat) - val bOpt = yMean - model.wOpt.transpose().mmul(colMeanMat.div(colSdMat)).get(0) - - new RidgeRegressionModel(wOpt, bOpt, model.lambdaOpt, model.lambdas) - } - - def main(args: Array[String]) { if (args.length != 2) { println("Usage: RidgeRegression ") System.exit(1) } val sc = new SparkContext(args(0), "RidgeRegression") - val data = RidgeRegressionGenerator.loadData(sc, args(1)) + val data = MLUtils.loadData(sc, args(1)) val model = train(data, 0, 1000) sc.stop() } diff --git a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala index ff8640bb50..e6d3e94787 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala @@ -8,28 +8,8 @@ import org.jblas.DoubleMatrix object RidgeRegressionGenerator { - // Helper methods to load and save data used for RidgeRegression - // Data format: - // , ... - // where , are feature values in Double and - // is the corresponding label as Double - def loadData(sc: SparkContext, dir: String) = { - val data = sc.textFile(dir).map{ line => - val parts = line.split(",") - val label = parts(0).toDouble - val features = parts(1).trim().split(" ").map(_.toDouble) - (label, features) - } - data - } - - private def saveData(data: RDD[(Double, Array[Double])], dir: String) { - val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) - dataStr.saveAsTextFile(dir) - } - def main(args: Array[String]) { - if (args.length != 2) { + if (args.length != 5) { println("Usage: RidgeRegressionGenerator " + " ") System.exit(1) @@ -68,7 +48,7 @@ object RidgeRegressionGenerator { } } - saveData(data, outputPath) + MLUtils.saveData(data, outputPath) sc.stop() } } diff --git a/ml/src/main/scala/spark/ml/Updater.scala b/ml/src/main/scala/spark/ml/Updater.scala new file mode 100644 index 0000000000..bacb059377 --- /dev/null +++ b/ml/src/main/scala/spark/ml/Updater.scala @@ -0,0 +1,16 @@ +package spark.ml + +import org.jblas.DoubleMatrix + +abstract class Updater extends Serializable { + def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + (DoubleMatrix, Double) +} + +class SimpleUpdater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + (DoubleMatrix, Double) = { + val normGradient = gradient.mul(stepSize / math.sqrt(iter)) + (weightsOld.sub(normGradient), 0) + } +} diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala index cc46602edd..50ef472075 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -29,10 +29,10 @@ class RidgeRegressionSuite extends FunSuite { val testRDD = sc.parallelize(testData, 2) val model = RidgeRegression.train(testRDD, 0, 10) - assert(model.bOpt >= 2.9 && model.bOpt <= 3.1) - assert(model.wOpt.length === 2) - assert(model.wOpt.get(0) >= 0.9 && model.wOpt.get(0) <= 1.1) - assert(model.wOpt.get(1) >= 0.9 && model.wOpt.get(1) <= 1.1) + assert(model.intercept >= 2.9 && model.intercept <= 3.1) + assert(model.weights.length === 2) + assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) + assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) sc.stop() // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown From 48770419bdc09b2cf1ddef3cbdd2c1d23f1748ac Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 10 Jun 2013 23:17:06 -0700 Subject: [PATCH 104/419] Add random data used for LR testing. Verified that results match with glm in R --- ml/data/lr-data/random.data | 1000 +++++++++++++++++++++++++++++++++++ 1 file changed, 1000 insertions(+) create mode 100755 ml/data/lr-data/random.data diff --git a/ml/data/lr-data/random.data b/ml/data/lr-data/random.data new file mode 100755 index 0000000000..29bcb8acba --- /dev/null +++ b/ml/data/lr-data/random.data @@ -0,0 +1,1000 @@ +0.0,-0.19138793197590276 0.7834675900121327 +1.0,3.712420417753061 3.55967640829891 +0.0,-0.3173743619974614 0.9034702789806682 +1.0,4.759494447180777 3.407011867344781 +0.0,-0.7078607074437426 -0.7866705652344417 +1.0,2.6708084832010215 2.5322909406378016 +0.0,-0.07553885038446313 -0.1297104483563081 +1.0,2.759487072285262 2.474689814713741 +0.0,-2.2199161547238107 0.7543109438660762 +1.0,1.922617509832946 1.9412373902594937 +0.0,0.8140942462004225 1.883920822277784 +1.0,1.7649295902120172 3.8195077526061363 +0.0,-1.1173052428096684 -1.468964723960145 +1.0,1.8733449544967458 2.913026590975709 +0.0,-0.11212965215910947 1.068087981775071 +1.0,2.3368459971730227 5.453870208593922 +0.0,-1.2802488543364463 -0.47218504171867676 +1.0,4.1917343620336895 3.5602286778418355 +0.0,0.5995976502137177 -0.797374550890321 +1.0,3.721592294428238 4.824418090974808 +0.0,-0.0721649164244053 -1.3952880192542576 +1.0,3.609764030146346 3.4730043476891277 +0.0,-1.5078269860498976 -2.6460421495665987 +1.0,1.8510254911824193 1.6748364225650059 +0.0,1.021485727769095 -0.14476425336866738 +1.0,4.10105000223134 2.3772502437548493 +0.0,2.6132710211418675 -1.061646527586342 +1.0,2.6444875273854653 4.043302750329545 +0.0,1.115723715938777 0.38401588153403887 +1.0,2.045759949164019 3.156447533448806 +0.0,-1.0543022640565405 -0.6820337845705753 +1.0,3.535337069948117 3.8121122972294965 +0.0,0.9427529503486505 -0.25123516319259886 +1.0,3.9611643301316795 3.3144121016644443 +0.0,-0.15013188927817916 0.8178862482229886 +1.0,3.200504584029051 2.3088398886136057 +0.0,0.819731993393585 -0.47386644109886344 +1.0,3.283317566020217 3.4828146842654513 +0.0,-2.3283941193793303 -0.6148925379529 +1.0,3.901670215294089 3.6356776610143324 +0.0,-0.28635769830042973 0.049586437072917544 +1.0,3.1114746381043927 3.6314805300338775 +0.0,-1.3085536069757229 0.11172767926766304 +1.0,3.3676979357140744 4.689661419564771 +0.0,-1.5820787210442733 1.3226576351191428 +1.0,2.5957586701668207 3.0648240201825923 +0.0,-2.116823743560968 0.272822309954307 +1.0,3.31672509500716 3.870172182480263 +0.0,0.09751166932653511 0.6469052579904877 +1.0,2.0609623373451305 3.9496181906908694 +0.0,0.5238217321419351 -1.2424816480725946 +1.0,3.5731384504449717 5.293293512805712 +0.0,-0.8507917425723299 -1.2243124053200718 +1.0,3.3060954421001867 3.1337045819604565 +0.0,1.5066706426420082 0.04176666807070882 +1.0,4.197316426430547 2.327643377792433 +0.0,-1.8068158696573955 -1.6380836149377855 +1.0,3.568239793850545 3.561688791420822 +0.0,0.4705756905309871 1.1991675114038487 +1.0,4.85003762884306 4.253420553408024 +0.0,0.7595792932847568 0.014062431397674205 +1.0,1.6984862661221896 1.7746925013882613 +0.0,0.1132294255888917 -0.09228036942051128 +1.0,3.766092539171029 2.765647342841482 +0.0,1.053401788561791 -1.0588667339849278 +1.0,2.780021685872393 3.239478188786074 +0.0,0.4042022490052266 1.0982210323828034 +1.0,2.4939569547402063 2.4615506964861273 +0.0,0.4469359967563411 0.3880418183993791 +1.0,2.7943749030887486 3.742182807141721 +0.0,-0.4418685162293727 0.802180923066725 +1.0,3.711213212127241 4.620177703831104 +0.0,0.10737314976605918 -1.5716142960765325 +1.0,4.0522289913808365 3.77562942835957 +0.0,1.4798827061781141 1.1638601205648005 +1.0,3.6758023575825547 3.115500589955362 +0.0,-1.803338141681238 -0.639996207387159 +1.0,2.044667029270621 3.04922768663927 +0.0,-0.06067427095346295 1.394611410740688 +1.0,4.626495834477846 2.995800202291488 +0.0,-0.2770274350630315 0.4521526506693692 +1.0,3.130857841268635 3.76858860814448 +0.0,2.163400739017478 -1.303601716798734 +1.0,2.9131896969824367 3.4288919990054167 +0.0,-0.7145108501670207 1.4189762494365543 +1.0,3.535768896041034 1.4894011726406373 +0.0,1.605614523747256 0.29974289519139824 +1.0,2.413678734728178 2.1826316767457183 +0.0,-0.8821932593373774 0.26432786248412726 +1.0,2.0878695933047116 3.5277388966365177 +0.0,-1.107001191509183 0.38421647065699477 +1.0,2.6462094774496454 2.273786785429519 +0.0,1.0712046043765102 -1.1889735666835115 +1.0,3.7458483094910666 1.3868020542832566 +0.0,-0.8403883736429167 -0.7163969561320671 +1.0,3.3359151000342195 3.2382001552279576 +0.0,0.13309387098922537 0.938761191821517 +1.0,2.083439571838502 3.2204948086228944 +0.0,1.3030219848568272 0.5976630914634896 +1.0,2.7602376200551317 2.200505791897739 +0.0,-0.9458633178207942 0.0490955863627428 +1.0,3.7998466026531883 1.9291683955712686 +0.0,-1.327236501803235 0.06915643957270164 +1.0,3.4740573335685925 2.1080735512507114 +0.0,0.8627688253416859 -1.961802291046532 +1.0,3.5108780392869776 3.9854745964798326 +0.0,-0.69537574439301 0.2436269580373554 +1.0,2.920286302932126 4.704192389485899 +0.0,-2.031190954684878 -0.7843052045579578 +1.0,1.6768848711259499 1.345658047606076 +0.0,0.9234894202027507 -0.38179572928866495 +1.0,3.1710339307651334 4.129874876536583 +0.0,-2.5086697007630376 -0.2638692986795807 +1.0,2.079400422215581 3.124756711992435 +0.0,-0.1388012859869782 0.3698243463601514 +1.0,2.665728164475424 4.574860576068532 +0.0,0.11967116650891912 -0.8792117975750646 +1.0,3.042630437105455 2.7245525508413677 +0.0,0.6078023848042808 -0.7977233104047035 +1.0,3.3340709038589638 4.962729210819017 +0.0,0.6373101353982795 1.1335021278327686 +1.0,3.3821397455119446 4.349379573895378 +0.0,-0.9140176931412027 -0.03428220013900756 +1.0,4.579963977595727 3.8322809335521484 +0.0,-0.43958506434874983 0.21259366700539037 +1.0,2.644701808902675 3.945416465403505 +0.0,-1.119921743746522 -0.2089105317801997 +1.0,2.5480553203091922 3.123344220515146 +0.0,0.8723990414181355 1.11150972420879 +1.0,4.479600967837827 2.8645066949820057 +0.0,-0.003869320481891422 0.24756134775982133 +1.0,3.237294368758498 4.642548547098718 +0.0,0.34643329685515545 0.029869480691029456 +1.0,2.6324740490008893 1.2577448307260846 +0.0,-0.4416403319035849 -1.4597062027342758 +1.0,1.764049052224297 3.649850384544675 +0.0,0.6779287737716254 -1.9489876700506967 +1.0,1.4286669812409405 2.4906452014102416 +0.0,-1.2271599940693638 0.9869686407012563 +1.0,3.6244117441765993 2.36879554315985 +0.0,-0.11422653411940642 0.4741905017884626 +1.0,3.6192153991840694 2.149436181779614 +0.0,0.45425900443207484 -1.357987041493406 +1.0,4.312295702128074 3.7596991900930252 +0.0,-0.35153502234686884 -0.6297451691082592 +1.0,3.4901363450669476 2.0630236379093243 +0.0,-1.5343533005821828 -0.23745688647461852 +1.0,4.775056734905926 5.291243824646301 +0.0,-1.032123659747431 0.8458711875294105 +1.0,2.3091889606097844 3.3688150059111215 +0.0,0.7854236849909306 0.6742463927844289 +1.0,3.284779531346899 2.855746734955609 +0.0,0.380579394855332 -1.2378905330462027 +1.0,2.540193014555953 3.245568950444961 +0.0,-0.5491810448400926 -2.3179482776107894 +1.0,3.481785462949587 1.8870182253717969 +0.0,-0.06833732101790825 2.178923334945784 +1.0,1.1663083809702222 1.8919272314310458 +0.0,-0.7801536433937879 -1.4185984368350903 +1.0,1.457713814592066 3.0323739348144048 +0.0,-0.16377716798970973 0.09678021896691058 +1.0,2.2294515799173094 1.6179126855486068 +0.0,-0.5845552895984718 -0.8095679531228397 +1.0,2.024328902209618 2.4660315284543888 +0.0,0.2037503424802764 1.5767438723426828 +1.0,3.5058983262252643 3.292836693091364 +0.0,-1.4004772080893082 0.6150928060180622 +1.0,4.610936499146778 3.3674445809820313 +0.0,-0.7325641160695897 -3.0469742419403225 +1.0,2.6778956983269926 4.049681967443553 +0.0,-0.3375932473421461 -0.32976087151423067 +1.0,3.975838378562512 1.2032482992228626 +0.0,-1.6622711226380826 -0.6954676646542216 +1.0,3.1601568512397256 2.7472491112914357 +0.0,0.6739969973916968 1.3608866192945286 +1.0,3.097978499063888 3.88429576456391 +0.0,-0.16445244300279913 0.631410854999902 +1.0,4.244875698991619 3.0464568222900477 +0.0,0.1749522197766453 -0.3295077792829936 +1.0,4.158913950688044 1.1836177376726964 +0.0,-1.8286320279969996 -0.6355826362111864 +1.0,2.4795264391445326 0.8073937061906746 +0.0,-0.5095499320702017 -0.8451757050184052 +1.0,3.6489546081475206 2.7405880916534957 +0.0,-0.11733097334574003 0.020300758125140466 +1.0,1.9034123919197892 4.036941742254072 +0.0,-0.4678304671259669 -0.7653895561277071 +1.0,2.555027220737054 4.205906511993216 +0.0,0.1952150967011765 1.2402178923240337 +1.0,3.532371144429582 2.395018092924601 +0.0,1.4682834110821084 2.2292327929025078 +1.0,2.1160331256749663 3.7157102308564824 +0.0,1.3973790173654674 -1.1902799121683607 +1.0,3.4775573554170616 3.0459058509488557 +0.0,-2.215337088722839 0.7693588032777773 +1.0,2.3298220860458976 1.5924630285528396 +0.0,1.260641664088144 1.5474089692944746 +1.0,4.460878990061944 2.595950219349794 +0.0,-1.8214944389802914 -1.9733205363211535 +1.0,4.41874870213851 2.4975116019313264 +0.0,1.2037921250123007 -0.7057578432831773 +1.0,3.042628088030598 3.7366256492570136 +0.0,-0.02609770715133313 -0.01975791007372346 +1.0,1.123824442324706 3.5115607224884466 +0.0,0.3466005704292144 -1.206858960323042 +1.0,3.044152779557358 2.4308738719304266 +0.0,-0.8292396838183249 -0.5768591341562801 +1.0,2.9898679252543325 3.3291086316901484 +0.0,0.6033357093153775 0.18738779274832332 +1.0,3.2777482224094916 2.2676548172839714 +0.0,-0.7104360487845565 -1.0365712508175688 +1.0,2.617802272534323 1.887796671556582 +0.0,-0.21008998836798706 -2.4424443035468957 +1.0,3.9387085143031317 2.368798316318223 +0.0,-0.65027380204969 0.4757828709083824 +1.0,1.6786020855223545 1.62019388696364 +0.0,0.40325101156361803 0.26629562725726075 +1.0,2.4614637796912167 2.778406744842399 +0.0,-0.4327374795655596 0.5643009301153851 +1.0,2.6419358755663103 2.1911675067034206 +0.0,-0.06058610052148417 0.6118154934715632 +1.0,4.134485645832481 4.214482766162727 +0.0,-2.091472947105952 -0.21279450874188077 +1.0,3.7664041746453503 0.5848083052756543 +0.0,0.20187441248519114 0.7310035835212488 +1.0,3.6821251396696817 1.2016937526237272 +0.0,0.16248871053987612 -0.8547163523143474 +1.0,3.1725037691095834 3.051265058839004 +0.0,-1.7466975308858639 -0.048497170816597705 +1.0,4.296665913992498 4.432036327276331 +0.0,-0.49371042139965376 -1.3162216335880739 +1.0,3.0767376272412292 2.4082404056282467 +0.0,0.6517145281009619 -0.15229289422910688 +1.0,3.8556129079007406 4.932746403550176 +0.0,2.467072616559744 -0.6570760874457315 +1.0,3.8722558954619446 2.398547361219584 +0.0,-0.996362973160808 -0.24663573264285635 +1.0,2.058960472055059 0.09020868936476445 +0.0,1.1921444033047794 -1.2205820383864918 +1.0,3.499255855340612 4.26015377680707 +0.0,0.46495431359796363 -0.3535071804767937 +1.0,3.2772715993311534 1.8496849599545144 +0.0,0.9200766227075026 1.0153595739730128 +1.0,3.7395665378166516 4.161859093428991 +0.0,-1.3445731221950805 0.3711182438638966 +1.0,1.974184816991473 2.3758202020218637 +0.0,0.25747673028745044 1.4898729695115611 +1.0,3.643667737073963 2.5171980898063024 +0.0,-0.7491175934837044 1.807998586131331 +1.0,3.024294668483263 2.745713910567566 +0.0,-2.9902104324990075 0.48847563269083094 +1.0,2.693457241550706 4.067192099378729 +0.0,1.0010822910854564 1.065617155304199 +1.0,2.6231328305267576 3.2530925652040796 +0.0,-1.569524799794976 0.10080365850268516 +1.0,5.543177898986999 3.149276748958176 +0.0,-0.2697035609845456 -0.3834981890675749 +1.0,5.5737716796876935 3.134627621089238 +0.0,0.16848836970122472 1.7680681560270155 +1.0,2.984578320659214 3.8081853301923743 +0.0,2.00864307305994 -1.1769936806590435 +1.0,2.4301644281026538 1.5357007015355957 +0.0,-1.251515087462618 -1.0023388301407077 +1.0,2.7783106123714036 3.4753675099443138 +0.0,1.2067779830446301 -1.1138369735803868 +1.0,2.660559526103853 0.9246419639107195 +0.0,-0.2120078291751072 0.553871125085326 +1.0,3.2961674182984613 4.1840551114889655 +0.0,-1.7407002661640898 -0.13494920714243758 +1.0,2.61652747199719 2.606431158365525 +0.0,0.1810536358726569 -0.7041543708042312 +1.0,0.6618977487425206 4.43976232230529 +0.0,-1.1056190552516114 -0.26273698119076755 +1.0,3.245745718364984 0.9585399121419127 +0.0,0.451245033031027 0.3966692171364385 +1.0,0.7000962854359294 2.5787278270774685 +0.0,-0.20657738352563298 -0.3054434424581368 +1.0,2.194893094322135 1.2265276851138993 +0.0,1.6478689673866447 -1.2217538409516264 +1.0,2.6520153534620268 4.253943157694819 +0.0,-1.091459682813003 -1.5933476790183565 +1.0,2.381978388803204 2.5725801073346375 +0.0,-1.7089448316753346 -0.40058783295112843 +1.0,4.692976595302646 2.293610804758882 +0.0,-0.8154594160076379 0.9100123432125261 +1.0,1.8893957859271135 2.365552941116367 +0.0,1.4750445045587657 -0.5730495722105764 +1.0,4.627946484342315 4.01023129091373 +0.0,-0.5740578222548407 -0.9010801407945085 +1.0,1.1844352711236998 1.0077910117111921 +0.0,-1.1904557430938465 -0.972229300373332 +1.0,1.9514043869587852 2.6603232743467817 +0.0,-0.11744191317950421 1.8160954524210857 +1.0,2.796337014232012 3.45131164191957 +0.0,1.1908754571951825 1.37388641966138 +1.0,3.1347230127964805 3.4874636513372774 +0.0,1.4279445191621287 0.4142573535049987 +1.0,3.2845746999649457 2.942571828876143 +0.0,1.0418078095097314 -0.515727237947711 +1.0,3.0672407807876674 3.593602465858237 +0.0,0.1070041194341431 0.013584199138111364 +1.0,2.831124413123504 2.5083468687281196 +0.0,1.9088191143015583 1.1943157723052062 +1.0,2.888463730373365 3.8588231186101716 +0.0,0.3344825700647222 1.4902421889158837 +1.0,5.1805240354926285 2.347000348613805 +0.0,-0.14736761539184529 -1.3764336595247777 +1.0,4.945788020165247 4.520764535128319 +0.0,0.48089579766964224 -1.0406729486881927 +1.0,3.115699146536788 3.0271206455481905 +0.0,0.8816867514268375 -0.7885530518936628 +1.0,3.293642905051253 4.129500570671647 +0.0,0.021019117419869213 -1.0983625263034136 +1.0,3.4712873315273884 2.8896550248710255 +0.0,1.336463967380889 0.1782538924176004 +1.0,2.9674559623039674 2.1702990000666977 +0.0,-0.9137873001694705 -1.6488427315604255 +1.0,2.425720985355789 3.336546225859983 +0.0,-2.3622279944776245 0.33443034793657744 +1.0,3.557057454549674 0.9654984504665607 +0.0,0.4924227412613347 0.8572441753897001 +1.0,2.903599258175698 1.9821387894597133 +0.0,-0.562864152759892 -1.41025535274598 +1.0,2.621542267864135 3.0896861639721602 +0.0,-0.9659016052287058 1.8601390770202668 +1.0,2.73394050343452 1.5908844566159697 +0.0,0.316736908826005 0.2857224419323005 +1.0,2.3312567009140532 5.596694984859762 +0.0,0.3137619371424862 -0.1840942808000176 +1.0,3.857644883242267 1.7425846536145542 +0.0,-0.10204795362718587 3.253153279848385 +1.0,1.991635750012152 3.0091345292604816 +0.0,0.6187841242310289 0.9589700354301842 +1.0,2.9773010080735895 3.723750625441197 +0.0,-0.8890787476930039 0.6057780620635984 +1.0,3.2341068438464773 4.238588226643048 +0.0,-0.6100941277292691 -1.5125630779121992 +1.0,3.378840902739636 2.0705801293719017 +0.0,1.9736225258875286 1.725383750563661 +1.0,1.8874237286900284 3.9061132751393997 +0.0,-0.0823939289302894 1.8958431169469556 +1.0,1.5927855001333566 4.6310125064091965 +0.0,0.3112044157520983 -1.7878471816057036 +1.0,4.34881513764263 3.4693940014863784 +0.0,1.052103622850019 -0.16912252356217902 +1.0,3.167179956507673 2.8792495587252507 +0.0,0.16791453003538387 -0.8546142448164881 +1.0,3.0538805073215953 3.4494667407676842 +0.0,-0.9500475678227512 0.06998146933806365 +1.0,3.8909913837847467 2.6813428719208763 +0.0,-0.09976816220585052 -1.4875944011133129 +1.0,3.1791447205478742 4.424991854067018 +0.0,1.0999643223476656 -1.1200747827607145 +1.0,5.222367041159025 1.2015274537211948 +0.0,-0.2848179798736651 0.401703345435371 +1.0,3.92690552314874 0.5307127426832543 +0.0,-0.6771410319499919 -0.5806616553853885 +1.0,3.611779415106116 3.3322298911093533 +0.0,-1.359189339369671 -0.03773529290863042 +1.0,4.696002594470123 1.4346348756461187 +0.0,-1.0094856636150293 0.19687532044013809 +1.0,3.2169383066148383 3.2307201581236473 +0.0,0.7836015359045666 0.2941037782687062 +1.0,3.7317041306588012 3.7985843457251107 +0.0,-0.3693168101963429 1.4513472421644549 +1.0,4.398703283685875 2.654636797434109 +0.0,0.02043081741683321 0.20805199015337653 +1.0,2.324187503797731 3.8819865944906566 +0.0,1.671377007435211 1.3731572027338659 +1.0,4.534630721644852 1.1543799480085444 +0.0,-0.3253127279932509 -0.8285225286171498 +1.0,3.993821155042294 0.7056403589045206 +0.0,1.194500226045371 0.638917136862092 +1.0,2.72148063695256 3.858678264350294 +0.0,-0.1905653672336637 0.8969404368665279 +1.0,1.9587911397509248 3.937696894952624 +0.0,-1.1358853052995896 1.4443151501322575 +1.0,3.7551091652428026 2.475478572543473 +0.0,-0.9167034706173607 -1.7549316646340103 +1.0,1.4669571532496661 3.2025879996118567 +0.0,-0.9673112226998997 0.13104324478779786 +1.0,5.129589009385082 2.962228456981596 +0.0,-1.038791699676283 0.3394661925580474 +1.0,4.0067362767396055 3.7808733451013863 +0.0,0.4607763000001474 0.3165842402170894 +1.0,3.470781763864157 3.1917117382789906 +0.0,-1.0759836593672722 2.1677955321765423 +1.0,1.8061608083541592 2.1368201192592524 +0.0,0.18913968729195288 -0.6832055159990379 +1.0,2.222086435460701 2.462434683952491 +0.0,1.1697195016246194 -0.6482703204844716 +1.0,0.9469729137532825 2.564223951962673 +0.0,-0.2596612587018774 1.3675954564898984 +1.0,3.3498722540414603 2.8411678301395655 +0.0,0.15549061976540607 -0.8795816620250406 +1.0,3.2166810907529517 3.3909740833940147 +0.0,-0.27777898312342497 1.5708467895548373 +1.0,3.5590852623593734 3.022687446035052 +0.0,0.8854804450462548 -0.1674059547432505 +1.0,5.592380230543062 2.046846128948299 +0.0,-0.38403645419139704 -0.6879614453050698 +1.0,1.2059037878354082 3.1373448113023263 +0.0,-0.9332349591768346 0.3271191223126651 +1.0,2.6941262027196444 2.0016455336591275 +0.0,1.985628476449888 -1.720937514961405 +1.0,1.52678578836386 3.6524268651279113 +0.0,0.14930924959259012 0.3549736192569231 +1.0,2.5081810800507904 4.502494324423253 +0.0,1.3659157029970181 -1.4064298168920828 +1.0,2.8947698041280185 3.871692848909248 +0.0,-0.19002791703482588 0.8099829390725909 +1.0,3.0481549176670555 4.05245395484312 +0.0,-0.014729952199541938 0.43445426055411474 +1.0,3.0874888030440486 3.89317889717026 +0.0,0.9521743475193137 0.16292125350371375 +1.0,3.0564028575123805 3.150394468127784 +0.0,-2.5565867181635724 1.1693524400747453 +1.0,3.963399476624186 2.655863627219969 +0.0,2.0594134768376584 1.4326082874689938 +1.0,3.9415985004601524 4.816989711315565 +0.0,0.4986273362656531 -0.30506819506279537 +1.0,2.7697598834307633 2.0292290332215512 +0.0,-0.4716043983943112 1.4692631198715722 +1.0,3.4127279940145883 3.078218915501194 +0.0,-0.28649487641740207 -0.8009455078808752 +1.0,2.645854233845017 4.028461076417125 +0.0,-1.2333241385253426 -0.2850384355482007 +1.0,2.4938754741404976 1.3466482769013481 +0.0,0.6872021385233428 -0.5159203960430369 +1.0,3.136974388668967 1.69291587793452 +0.0,0.9532239280401443 2.619265789851879 +1.0,2.570576389986536 2.548658346643033 +0.0,-1.030037965987706 0.2814883160676786 +1.0,2.510605023939257 2.3227098241155213 +0.0,2.4171507836629256 1.245606490445435 +1.0,3.5520681299250985 0.7442734445298673 +0.0,1.1940577980770877 1.6319950123919318 +1.0,2.708933998825159 2.118496371335553 +0.0,0.26808250222082186 2.5727974909556437 +1.0,3.221534693193204 3.073316472650363 +0.0,-0.6915734756410544 0.25168141600713434 +1.0,1.839319878312068 1.765565689559382 +0.0,1.708990562782385 1.1196517028520787 +1.0,2.1942131633492643 3.733776318231434 +0.0,1.4884941762679373 -0.5221400677305167 +1.0,2.425026062564176 4.814343944240822 +0.0,-1.3572570451352999 0.04542725800519613 +1.0,3.211869589232063 0.01498355271713292 +0.0,1.6170759581287553 0.7420944718274473 +1.0,1.8096883146020295 1.2063063122336204 +0.0,0.8326608996906895 -0.9760063002065638 +1.0,3.60415819299222 3.905143144181063 +0.0,0.9709971797789466 -1.0644382680658016 +1.0,2.8104103693138778 3.5792951568581017 +0.0,-1.021059644329913 -0.25967578007654707 +1.0,2.4020556940935216 3.8705560506781826 +0.0,-2.704107564850001 -0.14300257306795375 +1.0,3.7681081908063643 2.5433599278958297 +0.0,-0.537043950598385 0.8892208622861 +1.0,3.894301374710518 2.76168141850308 +0.0,-0.8416385593366815 1.3377079857054535 +1.0,1.4560861866861152 1.9464951398785584 +0.0,0.8974462212548237 -0.9027814165394935 +1.0,2.848274393366227 4.089266410865265 +0.0,-1.9874388443190703 -2.0515326123686 +1.0,1.7443330286532606 5.182730816947559 +0.0,1.9345124573698136 0.15482916596109797 +1.0,3.730890742221753 3.4571088485293173 +0.0,-0.7591467032951466 0.7817400181511722 +1.0,1.9612060838774241 1.7874104906670758 +0.0,0.04241602781710118 1.7624663777014242 +1.0,2.983106574446788 2.057794179835603 +0.0,-2.2675373876565272 0.1810247094230928 +1.0,1.8242036739605434 3.2897838599534053 +0.0,0.42135250345103276 0.9201551657148959 +1.0,2.3324158301116547 3.2735600739611406 +0.0,-2.503382611181759 -0.604428052499623 +1.0,2.1068571110070753 1.3987709205712464 +0.0,-0.25006447102137164 1.1597904649452788 +1.0,3.6610503210650105 2.389802330720335 +0.0,0.6655774387829471 -0.7657689612002381 +1.0,3.85820287126228 5.653287382126853 +0.0,0.08244241317513575 0.4755361735454262 +1.0,3.6029514045048234 3.0483730792265247 +0.0,1.0276000901424318 -0.569237094330588 +1.0,2.484863163042475 3.4464671311141046 +0.0,0.24588867824456415 -0.7355421671684942 +1.0,2.8757627634577396 1.3730139621444188 +0.0,0.911649033206053 -1.0562220913143838 +1.0,0.6701966948829261 3.8815519088585195 +0.0,1.0649444423673609 0.5738944212075908 +1.0,3.1272553354329955 5.18450239514651 +0.0,-1.8305691156390467 -1.2811179644895232 +1.0,4.326027257587544 1.9589219729995737 +0.0,-0.2278417247639679 -0.6436775444106994 +1.0,3.9854139754166136 2.8662622299102947 +0.0,-0.33177487577648573 0.7122237484053809 +1.0,2.7631237758865255 2.490470927953921 +0.0,-0.2989203275224733 -0.9063254275476191 +1.0,2.7739570950234254 3.333596743208583 +0.0,-0.12025132003053318 -1.2251715775331837 +1.0,3.9028268386113307 2.580334438085556 +0.0,0.3114518803226873 0.35489645702286177 +1.0,2.8765994073916112 4.251640702192294 +0.0,-3.0895947568085367 -1.0526550179589378 +1.0,3.5182345295490216 2.764855512391279 +0.0,0.5749621254042305 0.7148834016467635 +1.0,4.039448299164001 2.377396087740471 +0.0,1.7077800661629936 -0.23711282974122355 +1.0,2.883211311171089 3.5259606315833287 +0.0,-1.0304518163976537 -0.16271910447066004 +1.0,3.8284470175501504 1.0841759781704199 +0.0,-1.3620621426919217 0.8678141368192274 +1.0,3.831976508070298 2.3592788803510505 +0.0,0.8398199934902235 0.8458121179021545 +1.0,2.166979759191688 4.408250411844058 +0.0,-1.2009412161006234 -0.04486968047943732 +1.0,3.0041897020427517 1.67577082931885 +0.0,-1.0550850035108499 2.6114061208535673 +1.0,1.46399823823424 3.6863318429400627 +0.0,-0.439942118867861 0.8107733517611471 +1.0,2.799907981207793 3.1021389011201244 +0.0,0.40512996190803663 -0.2720769110918539 +1.0,2.936414720731187 2.6121553148876706 +0.0,0.7864503163458285 0.879685137879171 +1.0,3.497848931993103 3.93953696354328 +0.0,1.0898800025299487 -0.3780987477521812 +1.0,3.0737866861658834 3.8281246288654067 +0.0,1.0100369320198321 -0.36412797089680377 +1.0,4.977156552398557 1.9361263628969327 +0.0,1.1948682006514484 -1.0421380659408503 +1.0,2.3707352395183743 3.319087891488442 +0.0,0.14662871945444525 -1.125277513770441 +1.0,4.18636170602371 5.079790109963499 +0.0,0.5213830491310841 2.5489667538554355 +1.0,3.456121838657517 2.9777488007628823 +0.0,1.3942157902546204 -0.7392170745991694 +1.0,4.027857416272539 2.5520251242493615 +0.0,0.6677437543225546 -0.7054702957392922 +1.0,2.419993627501343 3.147115729790262 +0.0,-1.1891285195785104 0.7121837556662985 +1.0,2.6768950566988114 2.746092902448666 +0.0,-0.5581632736462642 -0.8475377022167101 +1.0,2.2877649074222144 3.360822129377224 +0.0,0.12427410923130733 -0.029877611579596446 +1.0,2.1363649823278976 2.040672619624904 +0.0,0.164296403698455 -0.7853340225962958 +1.0,2.2867454265483063 2.920796736914219 +0.0,0.030938689766481568 0.02840531713718885 +1.0,4.935402862397514 4.984097800264938 +0.0,-0.49323021214001667 -0.009344009957387383 +1.0,2.2590589178865788 2.784700488476081 +0.0,-1.7996451721642797 -0.08927843209025701 +1.0,2.7189425454136047 3.366984002518318 +0.0,-0.4732503966611213 2.41667617281343 +1.0,1.914172722581019 2.723688261246487 +0.0,0.6854209215843875 -0.6321377274037409 +1.0,4.7025333481932705 2.6561807763401646 +0.0,0.016511529980536163 -0.4064291762993186 +1.0,1.3841179371371182 3.367159685928979 +0.0,-0.525665902025766 0.3189849885462113 +1.0,2.1237941386456276 3.4141040859263914 +0.0,-1.3977733609952327 1.6180332199555512 +1.0,3.3282228318571496 2.9879449742002184 +0.0,-1.3911999737510374 -0.47876736354905697 +1.0,3.071461319022103 3.902142645231827 +0.0,-1.4616870328596612 0.4234223737141411 +1.0,3.3069543201402576 1.3522887907099401 +0.0,0.1771175002160632 0.7092577154896049 +1.0,2.561517669553921 3.2663130772229185 +0.0,0.8635080818806004 1.7578935533355913 +1.0,3.3054989034355793 3.4205399612822633 +0.0,-0.5525474134214131 -0.008874526853035592 +1.0,5.024607965706471 3.377256085775693 +0.0,0.6499316691799448 0.7636813929956143 +1.0,1.7211648540475015 3.7290596058136307 +0.0,-0.4312096678787339 0.4723353140241522 +1.0,1.6269397815780402 1.9613109767814954 +0.0,0.06589250830042476 0.5659627954925366 +1.0,1.4141705667382305 2.9411215895612255 +0.0,-0.30655047441372724 1.134312621267185 +1.0,4.079371134159225 3.7127217011979767 +0.0,-0.11148410319718746 1.504423362990177 +1.0,3.21908765035085 1.5284527951297098 +0.0,0.38879874604519066 -0.7718569898512835 +1.0,3.0387686435299197 1.9571679686339727 +0.0,0.0432538958325193 -0.609046739618082 +1.0,3.858513576900389 2.3343789318227595 +0.0,-1.594606569379673 2.0291869081775498 +1.0,4.418575803606943 3.634284954659144 +0.0,-1.5657043498774568 0.48528442006547645 +1.0,3.7474369990653518 2.417108621170513 +0.0,-0.4087178618516316 -0.5585629524971241 +1.0,2.8830052178069345 2.714807180476644 +0.0,1.0200529614238536 1.633454495011907 +1.0,2.161101444560085 2.722233198993495 +0.0,0.8905571055499505 0.3531260808046299 +1.0,1.5770402091220281 2.5197577954902615 +0.0,0.19603489193696402 0.4391781215510938 +1.0,3.285302297900197 2.5981032583297274 +0.0,-1.7728311957227578 2.226646036588897 +1.0,2.212402423781055 2.994783519362575 +0.0,-0.26351331835428804 0.6197161896115081 +1.0,2.5101464936050144 2.747453537535198 +0.0,1.083443472210967 -0.7471502465676395 +1.0,2.618022142084275 3.201094589808021 +0.0,-0.10243507468644107 -1.5307780048431203 +1.0,2.0479014235932986 2.7174445598757764 +0.0,-0.2530316183327909 1.5105959457792464 +1.0,2.616239369128394 3.1011058356715644 +0.0,2.0703487677159997 -1.23039689097027 +1.0,2.00559575849234 3.088170264353322 +0.0,0.751453701775929 -0.34079600956200146 +1.0,2.6436129383324625 0.6934715851263205 +0.0,0.4735774669250165 0.24981500600111478 +1.0,3.614102521076285 3.297655445774221 +0.0,-0.8397190394129946 2.0791729859494583 +1.0,2.5800847823336372 2.312770726398467 +0.0,0.9528690775719402 -4.054641847252764 +1.0,1.6631425491523402 4.465488566725185 +0.0,-0.40442215938144854 2.1662912065078923 +1.0,3.2025444402071472 0.954639816329502 +0.0,0.8484611241529962 -0.6531501762867838 +1.0,2.907155165379039 4.494838051538261 +0.0,1.1473298350419248 -0.7604213061923158 +1.0,4.406872541176625 2.616395889868952 +0.0,-1.0643453307576694 0.32269083514118757 +1.0,3.4229771635424653 5.404174358063928 +0.0,0.8223012341648268 -2.0705983787489455 +1.0,0.6519219290294926 3.317297519573949 +0.0,0.6661739745821234 0.21368601256080724 +1.0,2.8092516816651187 2.9407143882873363 +0.0,-2.0396349059310626 0.6660958962860263 +1.0,1.621401319049101 2.120514741629026 +0.0,-0.6673242389540511 -1.033336539766657 +1.0,2.4729967381312257 2.0622671692969314 +0.0,0.318696287733599 0.7696143248064906 +1.0,-0.3310542190127661 2.503572170101248 +0.0,-0.024545405442632163 1.2826535279165514 +1.0,2.08361065329982 1.7709137020843035 +0.0,-0.03325908838419148 2.127731976717063 +1.0,0.8920712229737089 2.267227052639782 +0.0,2.4226620796703706 -1.5422597801969735 +1.0,2.6125707261695665 4.136941962252239 +0.0,0.710000430684373 -0.2365544035810329 +1.0,3.587983407259662 2.371118916918134 +0.0,1.548716105657387 2.6039797648647527 +1.0,2.288647833469394 2.8514285941696564 +0.0,0.5407956769257948 -1.4250712589214616 +1.0,3.9999271279969157 4.647262641336589 +0.0,0.46916438504363506 -0.16114805677977867 +1.0,3.9351714928555133 3.017851089635014 +0.0,-0.24683125971847 0.8686956304798523 +1.0,2.445900548419883 2.601998949302925 +0.0,0.9708272515136681 0.9540365110832763 +1.0,2.0889493306284472 1.670700190658552 +0.0,0.7573519355244429 -0.6731075400854291 +1.0,2.9938559890272676 0.5796453404844417 +0.0,-0.42350233780111274 0.1072223004754211 +1.0,3.22502989165533 3.2744724666391045 +0.0,-0.051171179793716125 0.035749085667007977 +1.0,4.256076524642883 3.956646576238979 +0.0,0.44715068158575316 -0.10904823199444005 +1.0,3.754239074295241 2.4862504435534283 +0.0,-0.12025734941101636 0.6682754649328633 +1.0,2.9673795614648815 3.6207880514009263 +0.0,-2.250093626462795 -0.49148713538228506 +1.0,1.7335315087131171 4.234455598757855 +0.0,-0.5145677322324603 -1.8872464244504652 +1.0,3.1524408905920547 2.534903833671654 +0.0,1.4188237424906527 -1.987300018397619 +1.0,3.025903676999244 2.1652631630581847 +0.0,0.5008343534015861 0.28011601768758965 +1.0,2.0039218613662197 2.3639397631018015 +0.0,1.342528231824729 1.0036076495884643 +1.0,3.3281244751369985 2.4251038991267277 +0.0,-0.38845861664115766 -1.5147629282596704 +1.0,2.613448357242925 4.463712912575443 +0.0,-0.19439583983218703 0.676381234314577 +1.0,1.0400516553104269 2.3981508685333424 +0.0,0.9469554018478826 -0.08144910777086176 +1.0,3.179705969662961 3.768848690124549 +0.0,0.39855441813668835 -1.6301847736954416 +1.0,2.1915941615815226 2.7947789889097763 +0.0,1.6023287643577222 0.05432794979410767 +1.0,1.5758610206949497 3.8709473262823777 +0.0,-1.3109119301269387 -0.8645189055395048 +1.0,3.715865055565244 1.9360512196442488 +0.0,-0.2073998491467907 -1.178882579876182 +1.0,2.565062666629786 2.3121370465462494 +0.0,-0.41397768670851737 -0.6674761320605563 +1.0,2.941938460212705 3.537877403937825 +0.0,0.5954231185191001 1.6839554319972647 +1.0,4.591360208911688 1.4381368838271187 +0.0,-1.3221878199013057 0.786799353955043 +1.0,0.6498018470693379 2.2143413646510095 +0.0,0.5346452265922554 0.45599002729248733 +1.0,2.668100742914233 2.679883986650412 +0.0,-0.22428284967184606 -1.0003823373608314 +1.0,4.233871998643562 3.3423521548333897 +0.0,0.7800144346305873 1.6512542456242612 +1.0,3.3192955924982677 4.664828345688715 +0.0,-0.9059493298933676 -0.42207747354389447 +1.0,3.1776956110847916 1.1393123509452483 +0.0,-0.5246202787832872 1.0246845701853746 +1.0,4.732113325540828 1.29018271893586 +0.0,0.9863596225434407 0.7506968948666005 +1.0,2.911409852038849 2.626474556246977 +0.0,0.8545346747310709 -2.1711133879380955 +1.0,2.476689592134109 4.03136160709651 +0.0,0.43108249592457043 0.4589971218864913 +1.0,3.2333287857145825 2.188137362144206 +0.0,1.4405649581445525 0.4131214094941824 +1.0,2.0631468420251093 3.807898318807702 +0.0,0.43964401099781425 0.6669437158150616 +1.0,2.165843657939062 4.109647016182597 +0.0,-0.9735452695016392 -0.6172105570335473 +1.0,3.169794653766589 3.2721053734106 +0.0,1.3129166037688875 -1.2040138532590103 +1.0,2.211361701514339 1.025981622029549 +0.0,0.3653350359702278 0.5229315457444437 +1.0,3.372206428302252 4.163685355869495 +0.0,-0.8690030167652726 0.3226849491596335 +1.0,4.188509026227427 2.1137749377457076 +0.0,2.2174789916979933 0.8249932442083762 +1.0,3.9224824525785706 2.9436443006575925 +0.0,0.1370905200148926 -0.043320354739616776 +1.0,3.1118662077850807 1.4983207834379917 +0.0,-0.5304073850344787 -0.4219778391981189 +1.0,1.2153552376808336 3.4749521622043438 +0.0,-2.545970043914331 -0.5480647959096547 +1.0,1.8097968872175412 4.733523163055134 +0.0,-0.5599306916727819 0.4648015112295201 +1.0,3.0242901796172204 4.354893518146392 +0.0,-0.49175893973189483 1.8635231981223406 +1.0,3.923889822736733 4.199324033436554 +0.0,0.32931083529824645 -1.2038529291812745 +1.0,2.8430570026355904 3.2581768028655214 +0.0,0.08015643729775149 -0.5281238499521005 +1.0,1.0251176552841985 2.452443183841665 +0.0,-1.4000614002792062 -0.4723026702712555 +1.0,4.642753244692533 3.5777684251625153 +0.0,-0.9732069449126244 -0.7507666182081589 +1.0,2.284811103731081 2.6226837934175817 +0.0,1.4938320459354653 1.2271703303402608 +1.0,2.5217907633717935 1.9804499278889345 +0.0,0.9177851256816916 -1.196945923903535 +1.0,2.650515007788954 0.9818159554114416 +0.0,-0.4172435945582116 0.11930551874205601 +1.0,1.8203127944592765 3.3069324017397594 +0.0,0.08195935202288789 -0.2585763476071969 +1.0,2.14910426585678 4.146147361847687 +0.0,1.578290774885182 0.16149960053586573 +1.0,1.2607405323635168 2.940350340912184 +0.0,1.6722138822230346 -0.5454073192477626 +1.0,0.3769561517619793 4.029314828130509 +0.0,-0.012008811772440746 0.2577932550827986 +1.0,2.330909580388283 3.1650439747088024 +0.0,-1.4224384024201595 -0.6369918128076046 +1.0,3.451178380794735 2.7553545272536746 +0.0,-0.7913135079702314 -0.012217405089490006 +1.0,3.7918310740082424 3.3927876820084033 +0.0,0.41016650792928255 0.3521369094279198 +1.0,2.380867149491576 3.7533007228820754 +0.0,-0.2787273586680994 1.3553543015884186 +1.0,2.8933236071325226 1.7975563396445144 +0.0,-0.4868680345968448 0.058461169788172784 +1.0,3.484434144626577 3.5622013162506683 +0.0,1.171904838026115 0.1162839888503951 +1.0,1.8132727587691455 2.238018140780368 +0.0,0.8114997821213137 -1.712768034302675 +1.0,2.977061410695451 2.802894970831404 +0.0,1.7141760742336318 0.5672102391229309 +1.0,3.2929421353515185 3.3754831695793945 +0.0,-2.280170614413754 -0.4912881923146271 +1.0,4.182771547422101 3.5331418354105812 +0.0,-0.2544453921577854 0.4682744998445509 +1.0,1.9236524545763007 2.628837510538455 +0.0,0.6645491524745186 -2.398604366119661 +1.0,3.50840713613987 3.7182332137428955 +0.0,-1.4532823239751684 -0.9916580822162051 +1.0,2.769613688635247 4.72661442603805 +0.0,-1.090104082054257 0.486265921887567 +1.0,3.4900626627065003 3.03025323652533 +0.0,1.4518716691137106 -0.10218738652959546 +1.0,2.745034544461333 4.366809709694589 +0.0,-0.17197050309086373 0.13673125942508174 +1.0,2.4934379443680985 2.954734256628178 +0.0,0.14078971520128297 -0.5401300324197861 +1.0,3.640563349517043 5.163454382169049 +0.0,1.0264020194022627 -0.8738489740165843 +1.0,3.791458514669831 2.2038333093620834 +0.0,-3.075231830613813 2.04054404065675 +1.0,4.647422323558612 3.5220753128741427 +0.0,-0.6423734479152313 0.5403500050100541 +1.0,1.5985339514690007 2.73447434771563 +0.0,-0.04474684215568748 -0.21477212224970194 +1.0,2.6701891009654792 3.9776885659794505 +0.0,-0.4714276238216119 1.4235807729101415 +1.0,3.5551789183755806 2.7057825768035104 +0.0,1.108254774651522 0.8596053056731966 +1.0,3.0623366138774983 2.718494058918926 +0.0,-1.375827910513567 0.011994162356159788 +1.0,3.841407434840553 2.8434319292302304 +0.0,-0.7149712282755271 0.1811986378283469 +1.0,5.155524316715826 2.1468464150279747 +0.0,-0.06822014690491127 -0.15801546435311806 +1.0,3.4838423066641173 4.211572262022802 +0.0,1.455177312877137 -0.9388697017811595 +1.0,3.917344840727481 3.569507254920478 +0.0,-2.080636526173827 -1.2489913979804321 +1.0,4.904327940183608 3.4289745068714295 +0.0,-1.4744723958060084 0.2930577753686633 +1.0,2.810346752831796 2.4062885063635333 +0.0,-0.17365054648101302 -2.26263747840141 +1.0,4.077713960215311 3.841309768575811 +0.0,1.581178479362914 -0.9672846912018417 +1.0,4.516244757634386 2.9078781629204054 +0.0,-1.5890391289381882 -0.4092245513024253 +1.0,3.359480708344044 3.7375262649030123 +0.0,1.5675385032786122 0.9010632060589036 +1.0,3.8564874267647644 3.060660915266198 +0.0,-0.2482500870678099 0.29655946916337894 +1.0,3.1672692968701397 1.1973226392521306 +0.0,-1.4471523637168304 0.5370395414503478 +1.0,4.814859889188941 2.229750617440331 +0.0,0.2812295731325761 0.6044036116090106 +1.0,2.4884527354338903 1.4171627784171204 +0.0,1.173099753717184 0.7948729712563257 +1.0,1.5092479631180256 4.1412277875509105 +0.0,-1.1453508695714685 -0.15567849492271865 +1.0,1.9397046305500465 3.430755367623314 +0.0,-1.6689604208958047 -1.161942047896626 +1.0,4.287905082572467 2.643797664646416 +0.0,0.5691715436318573 -0.6013793142266736 +1.0,2.622904412483301 1.769830678112635 +0.0,-1.0627706066421603 -1.2962746926911266 +1.0,2.5818494635089886 2.9547836545958663 +0.0,-1.555832778500785 0.6050365213516793 +1.0,0.6877755924513469 3.0627330470806617 +0.0,-0.6945984937358738 -0.5355659085722678 +1.0,3.631758943383 2.6990914911890194 +0.0,-0.10204034384758799 1.2650405538373874 +1.0,2.8618200471403488 2.7676923144816237 +0.0,-1.2337428464512885 -0.7151041760567872 +1.0,3.5209869997316807 3.280763138579491 +0.0,0.3700095159793621 -0.8614396246939711 +1.0,2.698616090611572 3.2205340189872795 +0.0,-0.8069663812258417 -0.07956402748767083 +1.0,2.929873320056276 4.030067053746698 +0.0,-1.2316919288622938 1.245687935224532 +1.0,2.9285679560367055 2.9682906465530783 +0.0,-0.3965578686363537 1.1748126835359254 +1.0,4.002714110052464 4.370338584188975 +0.0,-0.6084107635744659 -0.6092872315132073 +1.0,3.293912876563504 3.5843332356258464 +0.0,-0.8145032742370918 1.4050967895930515 +1.0,1.991600071099763 2.343264260750465 +0.0,-0.9433799779882722 1.5943129187456013 +1.0,2.369037146473894 1.9827898318071764 +0.0,-0.26885731570182714 0.47421918725401946 +1.0,3.263006333756187 3.0441051541001443 +0.0,0.21785408377528742 0.5754303556190559 +1.0,2.941128899266118 1.240818619804987 +0.0,0.736142634408259 -1.3173589352849961 +1.0,3.2027184783050644 2.9218716893221766 +0.0,1.9216539101612737 -2.2400666381338694 +1.0,2.4823406743823426 3.429705681271458 +0.0,0.0666674809216063 -0.976496437708073 +1.0,3.206108328915537 2.0828009180110976 +0.0,-0.11582094814525531 2.5093876016868366 +1.0,2.5373176496966328 2.32926952602907 +0.0,-0.9237765727032562 0.9342845305943139 +1.0,2.5300867778672123 3.2754703213122753 +0.0,0.13837351460348038 0.2533025702882705 +1.0,4.556185356940701 0.7629684714626066 +0.0,-1.8251759895063635 0.6966019254550819 +1.0,4.905392053322123 4.111245902434462 +0.0,0.09886105139472441 1.4093224263552915 +1.0,2.0484713074013223 4.874632770975326 +0.0,-0.040609033066195156 -1.3446008307073973 +1.0,3.678642687565624 4.156505531118834 +0.0,0.052003196801406706 1.2239229001362555 +1.0,3.4376496474012876 2.417529764306501 +0.0,-0.09054032070414311 -1.7571173217955876 +1.0,3.230032966809188 3.5965216835420546 +0.0,0.9100014718072797 0.5615698517199065 +1.0,3.938728443662248 3.2945250621813273 +0.0,-0.9205165004286314 -0.01425448590777016 +1.0,1.907285344344031 3.8629943281683987 +0.0,-0.8160057252300347 -0.2757475590440447 +1.0,2.3076630082503926 3.2283118851645476 +0.0,1.3000520665928303 0.581203895654615 +1.0,3.8425274250736887 3.6133028383400414 +0.0,0.13694776598217193 -1.1659103408047182 +1.0,2.688548985689179 1.5486856086329917 +0.0,-0.14378057635986438 -1.4649914115754739 +1.0,3.923705106138171 3.8281415874634783 +0.0,1.3334544187579878 -0.048721556115349604 +1.0,3.320777445436592 2.947489296620178 +0.0,-0.36251547004650103 -0.2886015741883188 +1.0,3.2163584307843567 2.9285953038088373 +0.0,0.5437339741631225 -0.23459273264636704 +1.0,2.820666118654177 4.0305429519659395 +0.0,0.04808393980018175 0.42285718084497675 +1.0,1.4686721107589078 2.6605885841423067 +0.0,1.1873828480862414 0.5487600196906772 +1.0,3.425690422789916 4.252827757634791 +0.0,-0.7323210179394448 -0.9818194354330615 +1.0,3.018263609974841 2.914037267945018 +0.0,1.005159548514262 -0.5055899932767433 +1.0,4.566046579419102 5.545663797862058 +0.0,-0.7129346827436536 2.2938920919917742 +1.0,2.869336979055624 2.5688122980246684 +0.0,1.5201806096451054 -0.7414084378784415 +1.0,1.71558426191034 2.4576286538624794 +0.0,0.8090326808020629 0.26208059965589425 +1.0,3.0163716479573077 2.4747608384001056 +0.0,0.47627288733283857 1.3085076289292734 +1.0,3.3891272567835684 3.20832981462489 +0.0,1.0488767400026389 1.2318533170755142 +1.0,3.3428160616141853 2.5497426855885075 +0.0,-0.6411040361810151 -0.4290410178863531 +1.0,2.219119637941564 2.6621113083439254 +0.0,1.5621125506487947 0.7273124535333745 +1.0,3.1459765929197636 1.3663869759433418 +0.0,-0.05263982623034547 0.43675636434345644 +1.0,1.890191705836878 3.435071392429276 +0.0,0.28718983621307775 -2.438042507707637 +1.0,5.717207001359904 2.2303522388797035 +0.0,0.17636841934036573 -0.2202348356695646 +1.0,2.7426941364254294 3.9506423829670734 +0.0,-1.118995077703066 0.6062681312772151 +1.0,4.510963440028501 2.4497214672006575 +0.0,0.07601426739661686 1.4712413920907517 +1.0,2.472822799411239 4.045939967967948 +0.0,-2.2061186560242603 0.32560701091997957 +1.0,3.250675248798315 3.268273446922124 +0.0,-0.024542349115316425 1.5505593308513355 +1.0,2.5654508852779654 2.9476923150082874 +0.0,0.8070230851041806 1.0614288963806608 +1.0,4.0121013342203655 1.7608333223695753 +0.0,-0.6895596222836047 0.035498410809669464 +1.0,1.697905057706837 4.053746875797327 +0.0,-0.3311042917990167 -0.09180266122060314 +1.0,3.720796880080382 4.467214289132983 +0.0,-0.318673057944378 -3.1474317710285202 +1.0,4.809204233917482 4.55250051737848 +0.0,0.596445093094233 0.41780789823963405 +1.0,4.432965399675368 3.4638105151117617 +0.0,-0.10285141484897965 1.747950423830727 +1.0,2.1513849154027014 3.9020766404442933 +0.0,1.5988780419195843 -0.08753929889987294 +1.0,0.9867334105272594 3.017081919852008 +0.0,-1.4952194834476749 1.0187701527429442 +1.0,2.2468599817570376 2.5883807516977395 +0.0,-1.804930212071194 0.3519094744696904 +1.0,4.1524048686549975 2.39387437993355 +0.0,0.7077190974093445 0.5703893640810606 +1.0,3.551726989450847 2.4786821848615985 +0.0,1.866022101379231 0.23733176192158173 +1.0,2.636453843734601 3.2607059005922467 +0.0,1.0052825898444602 0.5988275134415102 +1.0,2.643754787324359 3.72363185525656 +0.0,-0.9925822461102075 0.060644514219670244 +1.0,3.8994350969658136 1.9246001662480055 +0.0,0.6513177047637154 0.04450296971216735 +1.0,2.4564101844841106 3.6785165656991596 +0.0,0.2606556093620563 -0.6172755504020078 +1.0,2.4170362032345674 0.8639272362396189 +0.0,-0.6416537078444019 1.8622433251026849 +1.0,2.0247632881021267 2.538336421666863 +0.0,-1.0177991501405648 -0.8522549981552515 +1.0,3.3426117902650185 3.1635532244875586 +0.0,-0.08963512689480763 1.4555128614393191 +1.0,3.7470117779591092 3.414476280017385 +0.0,0.7721815837750134 -0.17297061945116646 +1.0,3.823597567639877 4.2427688079492665 +0.0,-0.6905817293226868 0.5838402640342898 +1.0,3.005258204213709 2.7252310853631125 +0.0,0.963732273262942 -1.3950688358262504 +1.0,3.2803836447761934 3.448945851174787 +0.0,-0.11576488451784747 1.8796627145034757 +1.0,3.905782244273501 3.3853014175990412 +0.0,0.3786078767939069 0.4054987293824608 +1.0,4.251338642737948 3.2212804055347375 +0.0,1.785664685579919 -0.4528337660796719 +1.0,0.9522164714530392 4.648272724469027 +0.0,2.06805484281029 0.3211833348167774 +1.0,3.2063266406360875 3.20907719820361 +0.0,-0.18542396323311192 -0.4721814985954186 +1.0,1.2468417100913183 2.988063666542869 +0.0,-0.9089767150726245 0.049627884005341995 +1.0,3.570670591235201 1.812766580123238 +0.0,1.9973417232460495 -0.17709723581574177 +1.0,2.810527831677345 2.0292239826226717 +0.0,0.06390562956663569 0.9110683296487658 +1.0,4.449308253046676 2.5895593413305997 +0.0,-0.18596846882351442 1.2495641818989083 +1.0,2.1189215966743986 3.7928094437779283 From fd137bd7c6b924dfb818b8ff45b9c6bec0d47700 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 12 Jun 2013 16:30:04 -0700 Subject: [PATCH 105/419] Address Reynold's comments. Also use a builder pattern to construct the regression classes. --- ml/src/main/scala/spark/ml/Gradient.scala | 19 +++-- .../main/scala/spark/ml/GradientDescent.scala | 21 ++++- .../scala/spark/ml/LogisticRegression.scala | 80 ++++++++++++++---- ml/src/main/scala/spark/ml/MLUtils.scala | 22 +++-- ml/src/main/scala/spark/ml/Regression.scala | 84 +++++++++++-------- .../main/scala/spark/ml/RidgeRegression.scala | 75 +++++++++++++---- ml/src/main/scala/spark/ml/Updater.scala | 15 +++- 7 files changed, 233 insertions(+), 83 deletions(-) diff --git a/ml/src/main/scala/spark/ml/Gradient.scala b/ml/src/main/scala/spark/ml/Gradient.scala index a7e8327133..fe36187000 100644 --- a/ml/src/main/scala/spark/ml/Gradient.scala +++ b/ml/src/main/scala/spark/ml/Gradient.scala @@ -3,6 +3,13 @@ package spark.ml import org.jblas.DoubleMatrix abstract class Gradient extends Serializable { + /** + * Compute the gradient for a given row of data. + * + * @param data - One row of data. Row matrix of size 1xn where n is the number of features. + * @param label - Label for this data item. + * @param weights - Column matrix containing weights for every feature. + */ def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): (DoubleMatrix, Double) } @@ -14,11 +21,13 @@ class LogisticGradient extends Gradient { val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label val gradient = data.mul(gradientMultiplier) - val loss = if (margin > 0) { - math.log(1 + math.exp(0 - margin)) - } else { - math.log(1 + math.exp(margin)) - margin - } + val loss = + if (margin > 0) { + math.log(1 + math.exp(0 - margin)) + } else { + math.log(1 + math.exp(margin)) - margin + } + (gradient, loss) } } diff --git a/ml/src/main/scala/spark/ml/GradientDescent.scala b/ml/src/main/scala/spark/ml/GradientDescent.scala index 80d4c44a13..a03285bd53 100644 --- a/ml/src/main/scala/spark/ml/GradientDescent.scala +++ b/ml/src/main/scala/spark/ml/GradientDescent.scala @@ -10,6 +10,20 @@ import scala.collection.mutable.ArrayBuffer object GradientDescent { + /** + * Run gradient descent in parallel using mini batches. + * + * @param data - Input data for SGD. RDD of form (label, [feature values]). + * @param gradient - Gradient object that will be used to compute the gradient. + * @param updater - Updater object that will be used to update the model. + * @param stepSize - stepSize to be used during update. + * @param numIters - number of iterations that SGD should be run. + * @param miniBatchFraction - fraction of the input data set that should be used for + * one iteration of SGD. Default value 1.0. + * + * @return weights - Column matrix containing weights for every feature. + * @return lossHistory - Array containing the loss computed for every iteration. + */ def runMiniBatchSGD( data: RDD[(Double, Array[Double])], gradient: Gradient, @@ -18,22 +32,23 @@ object GradientDescent { numIters: Int, miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { - val lossHistory = new ArrayBuffer[Double] + val lossHistory = new ArrayBuffer[Double](numIters) val nfeatures: Int = data.take(1)(0)._2.length val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction + // Initialize weights as a column matrix var weights = DoubleMatrix.ones(nfeatures) var reg_val = 0.0 for (i <- 1 to numIters) { - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42).map { + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { case (y, features) => val featuresRow = new DoubleMatrix(features.length, 1, features:_*) val (grad, loss) = gradient.compute(featuresRow, y, weights) (grad, loss) - }.reduce((a, b) => (a._1.add(b._1), a._2 + b._2)) + }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) lossHistory.append(lossSum / miniBatchSize + reg_val) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index 5647bf5a84..e8e4f96dd3 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -1,7 +1,6 @@ package spark.ml import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ import org.jblas.DoubleMatrix @@ -14,9 +13,9 @@ class LogisticRegressionModel( val losses: Array[Double]) extends RegressionModel(weights, intercept) { override def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map { x => - val margin = (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept - 1.0/(1.0 + math.exp(margin * -1)) + test_data.map { x => + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) } } } @@ -25,9 +24,9 @@ class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends Regress override def normalizeData() = { // Shift only the features for LogisticRegression data.map { case(y, features) => - val featuresNormalized = (0 until nfeatures).map( - column => (features(column) - xColMean(column)) / xColSd(column) - ).toArray + val featuresNormalized = Array.tabulate(nfeatures) { column => + (features(column) - xColMean(column)) / xColSd(column) + } (y, featuresNormalized) } } @@ -44,25 +43,41 @@ class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends Regress } } -object LogisticRegression extends Logging { - val STEP_SIZE = 1.0 - val MINI_BATCH_FRACTION = 1.0 +class LogisticRegression(stepSize: Double, miniBatchFraction: Double, numIters: Int) + extends Regression with Logging { - def train(input: RDD[(Double, Array[Double])], numIters: Int) = { + override def train(input: RDD[(Double, Array[Double])]): RegressionModel = { input.cache() val lrData = new LogisticRegressionData(input) val data = lrData.normalizeData() val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), STEP_SIZE, numIters, MINI_BATCH_FRACTION) - + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, numIters) + val computedModel = new LogisticRegressionModel(weights, 0, losses) val model = lrData.scaleModel(computedModel) logInfo("Final model weights " + model.weights) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(",")) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) model } +} + +/** + * Helper classes to build a LogisticRegression object. + */ +object LogisticRegression { + + /** + * Build a logistic regression object with default arguments: + * + * @param stepSize as 1.0 + * @param miniBatchFraction as 1.0 + * @param numIters as 100 + */ + def builder() = { + new LogisticRegressionBuilder(1.0, 1.0, 100) + } def main(args: Array[String]) { if (args.length != 3) { @@ -71,7 +86,42 @@ object LogisticRegression extends Logging { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadData(sc, args(1)) - val model = train(data, args(2).toInt) + val lr = LogisticRegression.builder() + .setStepSize(2.0) + .setNumIterations(args(2).toInt) + .build() + val model = lr.train(data) sc.stop() } } + +class LogisticRegressionBuilder(stepSize: Double, miniBatchFraction: Double, numIters: Int) { + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + new LogisticRegressionBuilder(step, this.miniBatchFraction, this.numIters) + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + new LogisticRegressionBuilder(this.stepSize, fraction, this.numIters) + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + new LogisticRegressionBuilder(this.stepSize, this.miniBatchFraction, iters) + } + + /** + * Build a Logistic regression object. + */ + def build() = { + new LogisticRegression(stepSize, miniBatchFraction, numIters) + } +} diff --git a/ml/src/main/scala/spark/ml/MLUtils.scala b/ml/src/main/scala/spark/ml/MLUtils.scala index eab5db61bd..d62198f426 100644 --- a/ml/src/main/scala/spark/ml/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/MLUtils.scala @@ -2,21 +2,27 @@ package spark.ml import spark.{RDD, SparkContext} +/** + * Helper methods to load and save data + * Data format: + * , ... + * where , are feature values in Double and is the corresponding label as Double. + */ object MLUtils { - // Helper methods to load and save data - // Data format: - // , ... - // where , are feature values in Double and - // is the corresponding label as Double - def loadData(sc: SparkContext, dir: String) = { - val data = sc.textFile(dir).map{ line => + /** + * @param sc SparkContext + * @param dir Directory to the input data files. + * @return An RDD of tuples. For each tuple, the first element is the label, and the second + * element represents the feature values (an array of Double). + */ + def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { + sc.textFile(dir).map { line => val parts = line.split(",") val label = parts(0).toDouble val features = parts(1).trim().split(" ").map(_.toDouble) (label, features) } - data } def saveData(data: RDD[(Double, Array[Double])], dir: String) { diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/Regression.scala index e19d58396b..26a3a05545 100644 --- a/ml/src/main/scala/spark/ml/Regression.scala +++ b/ml/src/main/scala/spark/ml/Regression.scala @@ -7,47 +7,61 @@ import spark.SparkContext._ import org.jblas.DoubleMatrix -abstract class RegressionModel( - val weights: DoubleMatrix, - val intercept: Double) { - +abstract class RegressionModel(val weights: DoubleMatrix, val intercept: Double) { def predict(test_data: RDD[Array[Double]]): RDD[Double] } abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Serializable { - var yMean: Double = 0.0 - var xColMean: Array[Double] = null - var xColSd: Array[Double] = null - var nfeatures: Int = 0 - var nexamples: Long = 0 + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() - // This will populate yMean, xColMean and xColSd - calculateStats() + val yMean: Double = data.map { case (y, features) => y }.reduce(_ + _) / nexamples - def normalizeData(): RDD[(Double, Array[Double])] - def scaleModel(model: RegressionModel): RegressionModel - - def calculateStats() { - this.nexamples = data.count() - this.nfeatures = data.take(1)(0)._2.length - - this.yMean = data.map { case (y, features) => y }.reduce(_ + _) / nexamples - - // NOTE: We shuffle X by column here to compute column sum and sum of squares. - val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => - val nCols = features.length - // Traverse over every column and emit (col, value, value^2) - (0 until nCols).map(i => (i, (features(i), features(i)*features(i)))) - }.reduceByKey { case(x1, x2) => - (x1._1 + x2._1, x1._2 + x2._2) + // NOTE: We shuffle X by column here to compute column sum and sum of squares. + private val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + Iterator.tabulate(nCols) { i => + (i, (features(i), features(i)*features(i))) } - val xColSumsMap = xColSumSq.collectAsMap() - - // Compute mean and unbiased variance using column sums - this.xColMean = (0 until nfeatures).map(x => xColSumsMap(x)._1 / nexamples).toArray - this.xColSd = (0 until nfeatures).map {x => - val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) - math.sqrt(v) - }.toArray + }.reduceByKey { case(x1, x2) => + (x1._1 + x2._1, x1._2 + x2._2) } + private val xColSumsMap = xColSumSq.collectAsMap() + + // Compute mean and unbiased variance using column sums + val xColMean: Array[Double] = Array.tabulate(nfeatures) { x => + xColSumsMap(x)._1 / nexamples + } + val xColSd: Array[Double] = Array.tabulate(nfeatures) { x => + val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) + math.sqrt(v) + } + + /** + * Normalize the provided input data. This function is typically called before + * training a classifier on the input dataset and should be used to center of scale the data + * appropriately. + * + * @return RDD containing the normalized data + */ + def normalizeData(): RDD[(Double, Array[Double])] + + /** + * Scale the trained regression model. This function is usually called after training + * to adjust the model based on the normalization performed before. + * + * @return Regression model that can be used for prediction + */ + def scaleModel(model: RegressionModel): RegressionModel +} + +trait Regression { + + /** + * Train a model on the provided input dataset. Input data is an RDD of (Label, [Features]) + * + * @return RegressionModel representing the model built. + */ + def train(input: RDD[(Double, Array[Double])]): RegressionModel } diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index c030223b85..f67fb45134 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -10,13 +10,16 @@ import org.jblas.Solve * Ridge Regression from Joseph Gonzalez's implementation in MLBase */ class RidgeRegressionModel( - weights: DoubleMatrix, - intercept: Double, - val lambdaOpt: Double, - val lambdas: List[(Double, Double, DoubleMatrix)]) extends RegressionModel(weights, intercept) { + weights: DoubleMatrix, + intercept: Double, + val lambdaOpt: Double, + val lambdas: List[(Double, Double, DoubleMatrix)]) + extends RegressionModel(weights, intercept) { - override def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map(x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept) + override def predict(test_data: RDD[Array[Double]]) = { + test_data.map { x => + (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + } } } @@ -24,9 +27,9 @@ class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends Regression override def normalizeData() = { data.map { case(y, features) => val yNormalized = y - yMean - val featuresNormalized = (0 until nfeatures).map( - column => (features(column) - xColMean(column)) / xColSd(column) - ).toArray + val featuresNormalized = Array.tabulate(nfeatures) { column => + (features(column) - xColMean(column)) / xColSd(column) + }.toArray (yNormalized, featuresNormalized) } } @@ -43,12 +46,9 @@ class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends Regression } } -object RidgeRegression extends Logging { - - def train(inputData: RDD[(Double, Array[Double])], - lambdaLow: Double = 0.0, - lambdaHigh: Double = 10000.0) = { +class RidgeRegression(lambdaLow: Double, lambdaHigh: Double) extends Regression with Logging { + def train(inputData: RDD[(Double, Array[Double])]): RegressionModel = { inputData.cache() val ridgeData = new RidgeRegressionData(inputData) val data = ridgeData.normalizeData() @@ -125,6 +125,22 @@ object RidgeRegression extends Logging { normModel } +} + +/** + * Helper classes to build a RidgeRegression object. + */ +object RidgeRegression { + + /** + * Build a RidgeRegression object with default arguments as: + * + * @param lowLambda as 0.0 + * @param hiLambda as 100.0 + */ + def builder() = { + new RidgeRegressionBuilder(0.0, 100.0) + } def main(args: Array[String]) { if (args.length != 2) { @@ -133,7 +149,36 @@ object RidgeRegression extends Logging { } val sc = new SparkContext(args(0), "RidgeRegression") val data = MLUtils.loadData(sc, args(1)) - val model = train(data, 0, 1000) + val ridgeReg = RidgeRegression.builder() + .setLowLambda(0) + .setHighLambda(1000) + .build() + + val model = ridgeReg.train(data) sc.stop() } } + +class RidgeRegressionBuilder(lowLambda: Double, hiLambda: Double) { + + /** + * Set the lower bound on binary search for lambda's. Default is 0. + */ + def setLowLambda(low: Double) = { + new RidgeRegressionBuilder(low, this.hiLambda) + } + + /** + * Set the upper bound on binary search for lambda's. Default is 100.0. + */ + def setHighLambda(hi: Double) = { + new RidgeRegressionBuilder(this.lowLambda, hi) + } + + /** + * Build a RidgeRegression object. + */ + def build() = { + new RidgeRegression(lowLambda, hiLambda) + } +} diff --git a/ml/src/main/scala/spark/ml/Updater.scala b/ml/src/main/scala/spark/ml/Updater.scala index bacb059377..3952008e28 100644 --- a/ml/src/main/scala/spark/ml/Updater.scala +++ b/ml/src/main/scala/spark/ml/Updater.scala @@ -3,13 +3,24 @@ package spark.ml import org.jblas.DoubleMatrix abstract class Updater extends Serializable { + /** + * Compute an updated value for weights given the gradient, stepSize and iteration number. + * + * @param weightsOld - Column matrix of size nx1 where n is the number of features. + * @param gradient - Column matrix of size nx1 where n is the number of features. + * @param stepSize - step size across iterations + * @param iter - Iteration number + * + * @return weightsNew - Column matrix containing updated weights + * @return reg_val - regularization value + */ def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): (DoubleMatrix, Double) } class SimpleUpdater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): - (DoubleMatrix, Double) = { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int): (DoubleMatrix, Double) = { val normGradient = gradient.mul(stepSize / math.sqrt(iter)) (weightsOld.sub(normGradient), 0) } From ed32ec2b3bf93b1090ade63f2be25c2deba890b9 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 12 Jun 2013 16:31:39 -0700 Subject: [PATCH 106/419] Update test based on interface changes --- ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala index 50ef472075..eb67974cf0 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -27,7 +27,12 @@ class RidgeRegressionSuite extends FunSuite { val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray val testRDD = sc.parallelize(testData, 2) - val model = RidgeRegression.train(testRDD, 0, 10) + val ridgeReg = RidgeRegression.builder() + .setLowLambda(0) + .setHighLambda(10) + .build() + + val model = ridgeReg.train(testRDD) assert(model.intercept >= 2.9 && model.intercept <= 3.1) assert(model.weights.length === 2) From 2d0e64900e798db6de5d6f7f725c372a3f20cdd1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 09:53:34 -0700 Subject: [PATCH 107/419] Convert regression classes to builder pattern. Remove extraneous methods and classes --- .../scala/spark/ml/LogisticRegression.scala | 77 ++++++++----------- .../main/scala/spark/ml/RidgeRegression.scala | 64 ++++++--------- .../scala/spark/ml/RidgeRegressionSuite.scala | 6 +- 3 files changed, 56 insertions(+), 91 deletions(-) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index e8e4f96dd3..ddc138cacf 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -43,9 +43,39 @@ class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends Regress } } -class LogisticRegression(stepSize: Double, miniBatchFraction: Double, numIters: Int) +class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Regression with Logging { + /** + * Construct a LogisticRegression object with default parameters + */ + def this() = this(1.0, 1.0, 100) + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + this.stepSize = step + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + this.numIters = iters + this + } + + override def train(input: RDD[(Double, Array[Double])]): RegressionModel = { input.cache() @@ -68,17 +98,6 @@ class LogisticRegression(stepSize: Double, miniBatchFraction: Double, numIters: */ object LogisticRegression { - /** - * Build a logistic regression object with default arguments: - * - * @param stepSize as 1.0 - * @param miniBatchFraction as 1.0 - * @param numIters as 100 - */ - def builder() = { - new LogisticRegressionBuilder(1.0, 1.0, 100) - } - def main(args: Array[String]) { if (args.length != 3) { println("Usage: LogisticRegression ") @@ -86,42 +105,10 @@ object LogisticRegression { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadData(sc, args(1)) - val lr = LogisticRegression.builder() + val lr = new LogisticRegression() .setStepSize(2.0) .setNumIterations(args(2).toInt) - .build() val model = lr.train(data) sc.stop() } } - -class LogisticRegressionBuilder(stepSize: Double, miniBatchFraction: Double, numIters: Int) { - - /** - * Set the step size per-iteration of SGD. Default 1.0. - */ - def setStepSize(step: Double) = { - new LogisticRegressionBuilder(step, this.miniBatchFraction, this.numIters) - } - - /** - * Set fraction of data to be used for each SGD iteration. Default 1.0. - */ - def setMiniBatchFraction(fraction: Double) = { - new LogisticRegressionBuilder(this.stepSize, fraction, this.numIters) - } - - /** - * Set the number of iterations for SGD. Default 100. - */ - def setNumIterations(iters: Int) = { - new LogisticRegressionBuilder(this.stepSize, this.miniBatchFraction, iters) - } - - /** - * Build a Logistic regression object. - */ - def build() = { - new LogisticRegression(stepSize, miniBatchFraction, numIters) - } -} diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index f67fb45134..42b370d43b 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -46,7 +46,26 @@ class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends Regression } } -class RidgeRegression(lambdaLow: Double, lambdaHigh: Double) extends Regression with Logging { +class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) + extends Regression with Logging { + + def this() = this(0.0, 100.0) + + /** + * Set the lower bound on binary search for lambda's. Default is 0. + */ + def setLowLambda(low: Double) = { + this.lambdaLow = low + this + } + + /** + * Set the upper bound on binary search for lambda's. Default is 100.0. + */ + def setHighLambda(hi: Double) = { + this.lambdaHigh = hi + this + } def train(inputData: RDD[(Double, Array[Double])]): RegressionModel = { inputData.cache() @@ -127,21 +146,8 @@ class RidgeRegression(lambdaLow: Double, lambdaHigh: Double) extends Regression } } -/** - * Helper classes to build a RidgeRegression object. - */ object RidgeRegression { - /** - * Build a RidgeRegression object with default arguments as: - * - * @param lowLambda as 0.0 - * @param hiLambda as 100.0 - */ - def builder() = { - new RidgeRegressionBuilder(0.0, 100.0) - } - def main(args: Array[String]) { if (args.length != 2) { println("Usage: RidgeRegression ") @@ -149,36 +155,10 @@ object RidgeRegression { } val sc = new SparkContext(args(0), "RidgeRegression") val data = MLUtils.loadData(sc, args(1)) - val ridgeReg = RidgeRegression.builder() - .setLowLambda(0) - .setHighLambda(1000) - .build() + val ridgeReg = new RidgeRegression().setLowLambda(0) + .setHighLambda(1000) val model = ridgeReg.train(data) sc.stop() } } - -class RidgeRegressionBuilder(lowLambda: Double, hiLambda: Double) { - - /** - * Set the lower bound on binary search for lambda's. Default is 0. - */ - def setLowLambda(low: Double) = { - new RidgeRegressionBuilder(low, this.hiLambda) - } - - /** - * Set the upper bound on binary search for lambda's. Default is 100.0. - */ - def setHighLambda(hi: Double) = { - new RidgeRegressionBuilder(this.lowLambda, hi) - } - - /** - * Build a RidgeRegression object. - */ - def build() = { - new RidgeRegression(lowLambda, hiLambda) - } -} diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala index eb67974cf0..06b5b1ae31 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -27,10 +27,8 @@ class RidgeRegressionSuite extends FunSuite { val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray val testRDD = sc.parallelize(testData, 2) - val ridgeReg = RidgeRegression.builder() - .setLowLambda(0) - .setHighLambda(10) - .build() + val ridgeReg = new RidgeRegression().setLowLambda(0) + .setHighLambda(10) val model = ridgeReg.train(testRDD) From 6aadaf4d71e9ed95363638673bda48cecd483c03 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 13:17:20 -0700 Subject: [PATCH 108/419] Move normalization to MLUtils and remove Regression trait. --- .../scala/spark/ml/LogisticRegression.scala | 64 +++++++----------- ml/src/main/scala/spark/ml/MLUtils.scala | 46 +++++++++++++ ml/src/main/scala/spark/ml/Regression.scala | 43 ++---------- .../main/scala/spark/ml/RidgeRegression.scala | 66 +++++++------------ .../scala/spark/ml/RidgeRegressionSuite.scala | 1 + 5 files changed, 104 insertions(+), 116 deletions(-) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index ddc138cacf..76c4fe54c8 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -8,9 +8,9 @@ import org.jblas.DoubleMatrix * Logistic Regression using Stochastic Gradient Descent. */ class LogisticRegressionModel( - weights: DoubleMatrix, - intercept: Double, - val losses: Array[Double]) extends RegressionModel(weights, intercept) { + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends RegressionModel { override def predict(test_data: spark.RDD[Array[Double]]) = { test_data.map { x => @@ -20,31 +20,8 @@ class LogisticRegressionModel( } } -class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends RegressionData(data) { - override def normalizeData() = { - // Shift only the features for LogisticRegression - data.map { case(y, features) => - val featuresNormalized = Array.tabulate(nfeatures) { column => - (features(column) - xColMean(column)) / xColSd(column) - } - (y, featuresNormalized) - } - } - - override def scaleModel(m: RegressionModel) = { - val model = m.asInstanceOf[LogisticRegressionModel] - val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) - val colMeanMat = new DoubleMatrix(xColMean.length, 1, xColMean:_*) - - val weights = model.weights.div(colSdMat) - val intercept = -1.0 * model.weights.transpose().mmul(colMeanMat.div(colSdMat)).get(0) - - new LogisticRegressionModel(weights, intercept, model.losses) - } -} - class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) - extends Regression with Logging { + extends Logging { /** * Construct a LogisticRegression object with default parameters @@ -75,17 +52,27 @@ class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, va this } + def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val nexamples: Long = input.count() - override def train(input: RDD[(Double, Array[Double])]): RegressionModel = { - input.cache() + val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) + + // Shift only the features for LogisticRegression + val data = input.map { case(y, features) => + val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) + val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) + (y, featuresNormalized.toArray) + } - val lrData = new LogisticRegressionData(input) - val data = lrData.normalizeData() val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, numIters) + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + + val weightsScaled = weights.div(xColSd) + val intercept = -1.0 * weights.transpose().mmul(xColMean.div(xColSd)).get(0) + + val model = new LogisticRegressionModel(weightsScaled, intercept, losses) - val computedModel = new LogisticRegressionModel(weights, 0, losses) - val model = lrData.scaleModel(computedModel) logInfo("Final model weights " + model.weights) logInfo("Final model intercept " + model.intercept) logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) @@ -99,15 +86,14 @@ class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, va object LogisticRegression { def main(args: Array[String]) { - if (args.length != 3) { - println("Usage: LogisticRegression ") + if (args.length != 4) { + println("Usage: LogisticRegression ") System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadData(sc, args(1)) - val lr = new LogisticRegression() - .setStepSize(2.0) - .setNumIterations(args(2).toInt) + val lr = new LogisticRegression().setStepSize(args(2).toDouble) + .setNumIterations(args(3).toInt) val model = lr.train(data) sc.stop() } diff --git a/ml/src/main/scala/spark/ml/MLUtils.scala b/ml/src/main/scala/spark/ml/MLUtils.scala index d62198f426..b2361e5564 100644 --- a/ml/src/main/scala/spark/ml/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/MLUtils.scala @@ -1,6 +1,9 @@ package spark.ml import spark.{RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix /** * Helper methods to load and save data @@ -30,4 +33,47 @@ object MLUtils { dataStr.saveAsTextFile(dir) } + /** + * Utility function to compute mean and standard deviation on a given dataset. + * + * @param data - input data set whose statistics are computed + * @param nfeatures - number of features + * @param nexamples - number of examples in input dataset + * + * @return (yMean, xColMean, xColSd) - Tuple consisting of + * yMean - mean of the labels + * xColMean - Row vector with mean for every column (or feature) of the input data + * xColSd - Row vector standard deviation for every column (or feature) of the input data. + */ + def computeStats(data: RDD[(Double, Array[Double])], nfeatures: Int, nexamples: Long): + (Double, DoubleMatrix, DoubleMatrix) = { + val yMean: Double = data.map { case (y, features) => y }.reduce(_ + _) / nexamples + + // NOTE: We shuffle X by column here to compute column sum and sum of squares. + val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + Iterator.tabulate(nCols) { i => + (i, (features(i), features(i)*features(i))) + } + }.reduceByKey { case(x1, x2) => + (x1._1 + x2._1, x1._2 + x2._2) + } + val xColSumsMap = xColSumSq.collectAsMap() + + val xColMean = DoubleMatrix.zeros(nfeatures, 1) + val xColSd = DoubleMatrix.zeros(nfeatures, 1) + + // Compute mean and unbiased variance using column sums + var col = 0 + while (col < nfeatures) { + xColMean.put(col, xColSumsMap(col)._1 / nexamples) + val variance = + (xColSumsMap(col)._2 - (math.pow(xColSumsMap(col)._1, 2) / nexamples)) / (nexamples) + xColSd.put(col, math.sqrt(variance)) + col += 1 + } + + (yMean, xColMean, xColSd) + } } diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/Regression.scala index 26a3a05545..f23524bfa8 100644 --- a/ml/src/main/scala/spark/ml/Regression.scala +++ b/ml/src/main/scala/spark/ml/Regression.scala @@ -7,36 +7,17 @@ import spark.SparkContext._ import org.jblas.DoubleMatrix -abstract class RegressionModel(val weights: DoubleMatrix, val intercept: Double) { +trait RegressionModel { + /** + * Predict values for the given data set using the model trained. + * + * @param test_data RDD representing data points to be predicted + * @return RDD[Double] where each entry contains the corresponding prediction + */ def predict(test_data: RDD[Array[Double]]): RDD[Double] } abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Serializable { - val nfeatures: Int = data.take(1)(0)._2.length - val nexamples: Long = data.count() - - val yMean: Double = data.map { case (y, features) => y }.reduce(_ + _) / nexamples - - // NOTE: We shuffle X by column here to compute column sum and sum of squares. - private val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => - val nCols = features.length - // Traverse over every column and emit (col, value, value^2) - Iterator.tabulate(nCols) { i => - (i, (features(i), features(i)*features(i))) - } - }.reduceByKey { case(x1, x2) => - (x1._1 + x2._1, x1._2 + x2._2) - } - private val xColSumsMap = xColSumSq.collectAsMap() - - // Compute mean and unbiased variance using column sums - val xColMean: Array[Double] = Array.tabulate(nfeatures) { x => - xColSumsMap(x)._1 / nexamples - } - val xColSd: Array[Double] = Array.tabulate(nfeatures) { x => - val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) - math.sqrt(v) - } /** * Normalize the provided input data. This function is typically called before @@ -55,13 +36,3 @@ abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Se */ def scaleModel(model: RegressionModel): RegressionModel } - -trait Regression { - - /** - * Train a model on the provided input dataset. Input data is an RDD of (Label, [Features]) - * - * @return RegressionModel representing the model built. - */ - def train(input: RDD[(Double, Array[Double])]): RegressionModel -} diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 42b370d43b..36db2570d9 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -10,11 +10,11 @@ import org.jblas.Solve * Ridge Regression from Joseph Gonzalez's implementation in MLBase */ class RidgeRegressionModel( - weights: DoubleMatrix, - intercept: Double, + val weights: DoubleMatrix, + val intercept: Double, val lambdaOpt: Double, val lambdas: List[(Double, Double, DoubleMatrix)]) - extends RegressionModel(weights, intercept) { + extends RegressionModel { override def predict(test_data: RDD[Array[Double]]) = { test_data.map { x => @@ -23,31 +23,8 @@ class RidgeRegressionModel( } } -class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends RegressionData(data) { - override def normalizeData() = { - data.map { case(y, features) => - val yNormalized = y - yMean - val featuresNormalized = Array.tabulate(nfeatures) { column => - (features(column) - xColMean(column)) / xColSd(column) - }.toArray - (yNormalized, featuresNormalized) - } - } - - override def scaleModel(m: RegressionModel) = { - val model = m.asInstanceOf[RidgeRegressionModel] - val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) - val colMeanMat = new DoubleMatrix(xColMean.length, 1, xColMean:_*) - - val weights = model.weights.div(colSdMat) - val intercept = yMean - model.weights.transpose().mmul(colMeanMat.div(colSdMat)).get(0) - - new RidgeRegressionModel(weights, intercept, model.lambdaOpt, model.lambdas) - } -} - class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) - extends Regression with Logging { + extends Logging { def this() = this(0.0, 100.0) @@ -67,23 +44,29 @@ class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) this } - def train(inputData: RDD[(Double, Array[Double])]): RegressionModel = { - inputData.cache() - val ridgeData = new RidgeRegressionData(inputData) - val data = ridgeData.normalizeData() - val nfeatures: Int = ridgeData.nfeatures - val nexamples: Long = ridgeData.nexamples + def train(input: RDD[(Double, Array[Double])]): RidgeRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val nexamples: Long = input.count() + + val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) + + val data = input.map { case(y, features) => + val yNormalized = y - yMean + val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) + val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) + (yNormalized, featuresNormalized.toArray) + } // Compute XtX - Size of XtX is nfeatures by nfeatures val XtX: DoubleMatrix = data.map { case (y, features) => val x = new DoubleMatrix(1, features.length, features:_*) x.transpose().mmul(x) - }.reduce(_.add(_)) + }.reduce(_.addi(_)) // Compute Xt*y - Size of Xty is nfeatures by 1 val Xty: DoubleMatrix = data.map { case (y, features) => new DoubleMatrix(features.length, 1, features:_*).mul(y) - }.reduce(_.add(_)) + }.reduce(_.addi(_)) // Define a function to compute the leave one out cross validation error // for a single example @@ -134,15 +117,16 @@ class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) // Return the model which contains the solution - val trainModel = new RidgeRegressionModel(weights, 0.0, lambdaOpt, lambdas) - val normModel = ridgeData.scaleModel(trainModel) + val weightsScaled = weights.div(xColSd) + val intercept = yMean - (weights.transpose().mmul(xColMean.div(xColSd)).get(0)) + val model = new RidgeRegressionModel(weightsScaled, intercept, lambdaOpt, lambdas) - logInfo("RidgeRegression: optimal lambda " + normModel.lambdaOpt) - logInfo("RidgeRegression: optimal weights " + normModel.weights) - logInfo("RidgeRegression: optimal intercept " + normModel.intercept) + logInfo("RidgeRegression: optimal lambda " + model.lambdaOpt) + logInfo("RidgeRegression: optimal weights " + model.weights) + logInfo("RidgeRegression: optimal intercept " + model.intercept) logInfo("RidgeRegression: cross-validation error " + cverror) - normModel + model } } diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala index 06b5b1ae31..07da5081b3 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -27,6 +27,7 @@ class RidgeRegressionSuite extends FunSuite { val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray val testRDD = sc.parallelize(testData, 2) + testRDD.cache() val ridgeReg = new RidgeRegression().setLowLambda(0) .setHighLambda(10) From 3a6924cb8ff082eed0243b0684de88a858beee7d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 13:25:10 -0700 Subject: [PATCH 109/419] Clean up some comments. --- ml/src/main/scala/spark/ml/GradientDescent.scala | 1 + ml/src/main/scala/spark/ml/LogisticRegression.scala | 4 +--- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/ml/src/main/scala/spark/ml/GradientDescent.scala b/ml/src/main/scala/spark/ml/GradientDescent.scala index a03285bd53..564f89e5ee 100644 --- a/ml/src/main/scala/spark/ml/GradientDescent.scala +++ b/ml/src/main/scala/spark/ml/GradientDescent.scala @@ -12,6 +12,7 @@ object GradientDescent { /** * Run gradient descent in parallel using mini batches. + * Based on Matlab code written by John Duchi. * * @param data - Input data for SGD. RDD of form (label, [feature values]). * @param gradient - Gradient object that will be used to compute the gradient. diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index 76c4fe54c8..c134f8d77d 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -6,6 +6,7 @@ import org.jblas.DoubleMatrix /** * Logistic Regression using Stochastic Gradient Descent. + * Based on Matlab code written by John Duchi. */ class LogisticRegressionModel( val weights: DoubleMatrix, @@ -80,9 +81,6 @@ class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, va } } -/** - * Helper classes to build a LogisticRegression object. - */ object LogisticRegression { def main(args: Array[String]) { From 76acc9fe9d76c43ee063642db3ea2a9190d97eb6 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 14:15:03 -0700 Subject: [PATCH 110/419] Make regression arguments private and add method to predict one point --- .../scala/spark/ml/LogisticRegression.scala | 13 ++++++-- ml/src/main/scala/spark/ml/Regression.scala | 31 +++++-------------- .../main/scala/spark/ml/RidgeRegression.scala | 10 ++++-- 3 files changed, 24 insertions(+), 30 deletions(-) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index c134f8d77d..4dc883fbf8 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -13,15 +13,22 @@ class LogisticRegressionModel( val intercept: Double, val losses: Array[Double]) extends RegressionModel { - override def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map { x => + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + val margin = dataMat.mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) + } } -class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) +class LogisticRegression(private var stepSize: Double, private var miniBatchFraction: Double, + private var numIters: Int) extends Logging { /** diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/Regression.scala index f23524bfa8..9d7424d656 100644 --- a/ml/src/main/scala/spark/ml/Regression.scala +++ b/ml/src/main/scala/spark/ml/Regression.scala @@ -1,38 +1,21 @@ package spark.ml -import java.io._ - -import spark.{RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix +import spark.RDD trait RegressionModel { /** * Predict values for the given data set using the model trained. * - * @param test_data RDD representing data points to be predicted + * @param testData RDD representing data points to be predicted * @return RDD[Double] where each entry contains the corresponding prediction */ - def predict(test_data: RDD[Array[Double]]): RDD[Double] -} - -abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Serializable { + def predict(testData: RDD[Array[Double]]): RDD[Double] /** - * Normalize the provided input data. This function is typically called before - * training a classifier on the input dataset and should be used to center of scale the data - * appropriately. + * Predict values for a single data point using the model trained. * - * @return RDD containing the normalized data + * @param testData array representing a single data point + * @return Double prediction from the trained model */ - def normalizeData(): RDD[(Double, Array[Double])] - - /** - * Scale the trained regression model. This function is usually called after training - * to adjust the model based on the normalization performed before. - * - * @return Regression model that can be used for prediction - */ - def scaleModel(model: RegressionModel): RegressionModel + def predict(testData: Array[Double]): Double } diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 36db2570d9..628fc25933 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -16,14 +16,18 @@ class RidgeRegressionModel( val lambdas: List[(Double, Double, DoubleMatrix)]) extends RegressionModel { - override def predict(test_data: RDD[Array[Double]]) = { - test_data.map { x => + override def predict(testData: RDD[Array[Double]]): RDD[Double] = { + testData.map { x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept } } + + override def predict(testData: Array[Double]): Double = { + (new DoubleMatrix(1, testData.length, testData:_*).mmul(this.weights)).get(0) + this.intercept + } } -class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) +class RidgeRegression(private var lambdaLow: Double, private var lambdaHigh: Double) extends Logging { def this() = this(0.0, 100.0) From 6dd3a816c8d438e1d8beebe0a1cbf5d90ef6e3bc Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 15:11:21 -0700 Subject: [PATCH 111/419] Use a private constructor instead of private vars --- ml/src/main/scala/spark/ml/LogisticRegression.scala | 4 ++-- ml/src/main/scala/spark/ml/RidgeRegression.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index 4dc883fbf8..3417bfe882 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -27,8 +27,8 @@ class LogisticRegressionModel( } } -class LogisticRegression(private var stepSize: Double, private var miniBatchFraction: Double, - private var numIters: Int) +class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, + var numIters: Int) extends Logging { /** diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 628fc25933..3cbb0653c3 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -27,7 +27,7 @@ class RidgeRegressionModel( } } -class RidgeRegression(private var lambdaLow: Double, private var lambdaHigh: Double) +class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) extends Logging { def this() = this(0.0, 100.0) From 43b398db6a2008a9dc0343df0ec0b24f96fa65c1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 14 Jun 2013 16:13:12 -0700 Subject: [PATCH 112/419] Fix logistic regression to not center data. Also add a feature to get the intercept correct and test these using a small unit test. --- .../scala/spark/ml/LogisticRegression.scala | 17 ++---- .../spark/ml/LogisticRegressionSuite.scala | 54 +++++++++++++++++++ 2 files changed, 59 insertions(+), 12 deletions(-) create mode 100644 ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index 3417bfe882..14c237c5ed 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -61,23 +61,16 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { - val nfeatures: Int = input.take(1)(0)._2.length - val nexamples: Long = input.count() - - val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) - - // Shift only the features for LogisticRegression - val data = input.map { case(y, features) => - val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) - val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) - (y, featuresNormalized.toArray) + // Add a extra variable consisting of all 1.0's for the intercept. + val data = input.map { case (y, features) => + (y, Array(1.0, features:_*)) } val (weights, losses) = GradientDescent.runMiniBatchSGD( data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) - val weightsScaled = weights.div(xColSd) - val intercept = -1.0 * weights.transpose().mmul(xColMean.div(xColSd)).get(0) + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) val model = new LogisticRegressionModel(weightsScaled, intercept, losses) diff --git a/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala new file mode 100644 index 0000000000..d66c2c8b35 --- /dev/null +++ b/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala @@ -0,0 +1,54 @@ +package spark.ml + +import spark.SparkContext +import spark.SparkContext._ +import spark.Logging + +import org.apache.commons.math3.distribution.NormalDistribution +import org.scalatest.FunSuite + +class LogisticRegressionSuite extends FunSuite with Logging { + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val rnd = new NormalDistribution(0, 1) + rnd.reseedRandomGenerator(42) + + val sc = new SparkContext("local", "test") + val x1 = Array.fill[Double](nPoints)(rnd.sample()) + + val A = 2.0 + val B = -1.5 + + // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) + val unifRand = new scala.util.Random(45) + val rLogis = (0 until nPoints).map { i => + val u = unifRand.nextDouble() + math.log(u) - math.log(1.0-u) + } + + // y <- A + B*x + rlogis(100) + // y <- as.numeric(y > 0) + val y = (0 until nPoints).map { i => + val yVal = A + B * x1(i) + rLogis(i) + if (yVal > 0) 1.0 else 0.0 + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val lr = new LogisticRegression().setStepSize(1.0) + .setNumIterations(1000) + + val model = lr.train(testRDD) + + assert(model.weights.get(0) >= -1.60 && model.weights.get(0) <= -1.40) + assert(model.intercept >= 1.9 && model.intercept <= 2.1) + + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } +} From 39ed41652beaad63c03540411a51ed82c1126e6d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 15 Jun 2013 09:03:20 -0700 Subject: [PATCH 113/419] Move to regression, util and gradient packages --- ml/src/main/scala/spark/ml/{ => gradient}/Gradient.scala | 2 +- .../main/scala/spark/ml/{ => gradient}/GradientDescent.scala | 2 +- ml/src/main/scala/spark/ml/{ => gradient}/Updater.scala | 2 +- .../scala/spark/ml/{ => regression}/LogisticRegression.scala | 4 +++- .../ml/{ => regression}/LogisticRegressionGenerator.scala | 3 ++- ml/src/main/scala/spark/ml/{ => regression}/Regression.scala | 2 +- .../scala/spark/ml/{ => regression}/RidgeRegression.scala | 3 ++- .../spark/ml/{ => regression}/RidgeRegressionGenerator.scala | 3 ++- ml/src/main/scala/spark/ml/{ => util}/MLUtils.scala | 2 +- .../spark/ml/{ => regression}/LogisticRegressionSuite.scala | 2 +- .../spark/ml/{ => regression}/RidgeRegressionSuite.scala | 2 +- 11 files changed, 16 insertions(+), 11 deletions(-) rename ml/src/main/scala/spark/ml/{ => gradient}/Gradient.scala (97%) rename ml/src/main/scala/spark/ml/{ => gradient}/GradientDescent.scala (98%) rename ml/src/main/scala/spark/ml/{ => gradient}/Updater.scala (97%) rename ml/src/main/scala/spark/ml/{ => regression}/LogisticRegression.scala (97%) rename ml/src/main/scala/spark/ml/{ => regression}/LogisticRegressionGenerator.scala (95%) rename ml/src/main/scala/spark/ml/{ => regression}/Regression.scala (95%) rename ml/src/main/scala/spark/ml/{ => regression}/RidgeRegression.scala (98%) rename ml/src/main/scala/spark/ml/{ => regression}/RidgeRegressionGenerator.scala (96%) rename ml/src/main/scala/spark/ml/{ => util}/MLUtils.scala (99%) rename ml/src/test/scala/spark/ml/{ => regression}/LogisticRegressionSuite.scala (98%) rename ml/src/test/scala/spark/ml/{ => regression}/RidgeRegressionSuite.scala (98%) diff --git a/ml/src/main/scala/spark/ml/Gradient.scala b/ml/src/main/scala/spark/ml/gradient/Gradient.scala similarity index 97% rename from ml/src/main/scala/spark/ml/Gradient.scala rename to ml/src/main/scala/spark/ml/gradient/Gradient.scala index fe36187000..50795c01a2 100644 --- a/ml/src/main/scala/spark/ml/Gradient.scala +++ b/ml/src/main/scala/spark/ml/gradient/Gradient.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.gradient import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/GradientDescent.scala b/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala similarity index 98% rename from ml/src/main/scala/spark/ml/GradientDescent.scala rename to ml/src/main/scala/spark/ml/gradient/GradientDescent.scala index 564f89e5ee..8cd2a69a3a 100644 --- a/ml/src/main/scala/spark/ml/GradientDescent.scala +++ b/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.gradient import spark.{Logging, RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/main/scala/spark/ml/Updater.scala b/ml/src/main/scala/spark/ml/gradient/Updater.scala similarity index 97% rename from ml/src/main/scala/spark/ml/Updater.scala rename to ml/src/main/scala/spark/ml/gradient/Updater.scala index 3952008e28..27f03897b0 100644 --- a/ml/src/main/scala/spark/ml/Updater.scala +++ b/ml/src/main/scala/spark/ml/gradient/Updater.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.gradient import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala similarity index 97% rename from ml/src/main/scala/spark/ml/LogisticRegression.scala rename to ml/src/main/scala/spark/ml/regression/LogisticRegression.scala index 14c237c5ed..1508f6934a 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -1,6 +1,8 @@ -package spark.ml +package spark.ml.regression import spark.{Logging, RDD, SparkContext} +import spark.ml.gradient._ +import spark.ml.util.MLUtils import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala similarity index 95% rename from ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala rename to ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala index ff46cfa46d..1617eac205 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala @@ -1,6 +1,7 @@ -package spark.ml +package spark.ml.regression import spark.{RDD, SparkContext} +import spark.ml.util.MLUtils import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/regression/Regression.scala similarity index 95% rename from ml/src/main/scala/spark/ml/Regression.scala rename to ml/src/main/scala/spark/ml/regression/Regression.scala index 9d7424d656..4a20f513b7 100644 --- a/ml/src/main/scala/spark/ml/Regression.scala +++ b/ml/src/main/scala/spark/ml/regression/Regression.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.regression import spark.RDD diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala similarity index 98% rename from ml/src/main/scala/spark/ml/RidgeRegression.scala rename to ml/src/main/scala/spark/ml/regression/RidgeRegression.scala index 3cbb0653c3..16d146ac27 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -1,7 +1,8 @@ -package spark.ml +package spark.ml.regression import spark.{Logging, RDD, SparkContext} import spark.SparkContext._ +import spark.ml.util.MLUtils import org.jblas.DoubleMatrix import org.jblas.Solve diff --git a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala similarity index 96% rename from ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala rename to ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala index e6d3e94787..ac7f1e7320 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala @@ -1,6 +1,7 @@ -package spark.ml +package spark.ml.regression import spark.{RDD, SparkContext} +import spark.ml.util.MLUtils import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/MLUtils.scala b/ml/src/main/scala/spark/ml/util/MLUtils.scala similarity index 99% rename from ml/src/main/scala/spark/ml/MLUtils.scala rename to ml/src/main/scala/spark/ml/util/MLUtils.scala index b2361e5564..76e1bbd36d 100644 --- a/ml/src/main/scala/spark/ml/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/util/MLUtils.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.util import spark.{RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala rename to ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index d66c2c8b35..ce388ecc26 100644 --- a/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.regression import spark.SparkContext import spark.SparkContext._ diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala rename to ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala index 07da5081b3..6d5f13d6f5 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.regression import spark.SparkContext import spark.SparkContext._ From 05be233ce2716fe57cf44433d52734ded29e3506 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 15:58:01 +0200 Subject: [PATCH 114/419] Removed dependency on Apache Commons Math --- .../regression/LogisticRegressionGenerator.scala | 12 ++++++------ .../ml/regression/RidgeRegressionGenerator.scala | 12 ++++++------ .../ml/regression/LogisticRegressionSuite.scala | 11 ++++++----- .../ml/regression/RidgeRegressionSuite.scala | 16 ++++++++-------- project/SparkBuild.scala | 3 +-- 5 files changed, 27 insertions(+), 27 deletions(-) diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala index 1617eac205..6d37aad047 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala @@ -1,11 +1,12 @@ package spark.ml.regression +import scala.util.Random + +import org.jblas.DoubleMatrix + import spark.{RDD, SparkContext} import spark.ml.util.MLUtils -import org.apache.commons.math3.distribution.NormalDistribution -import org.jblas.DoubleMatrix - object LogisticRegressionGenerator { def main(args: Array[String]) { @@ -25,12 +26,11 @@ object LogisticRegressionGenerator { val sc = new SparkContext(sparkMaster, "LogisticRegressionGenerator") val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => - val rnd = new NormalDistribution(0, 1) - rnd.reseedRandomGenerator(42 + idx) + val rnd = new Random(42 + idx) val y = if (idx % 2 == 0) 0 else 1 val x = Array.fill[Double](nfeatures) { - rnd.sample() + (y * eps) + rnd.nextGaussian() + (y * eps) } (y, x) } diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala index ac7f1e7320..75854fe1de 100644 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala @@ -1,11 +1,12 @@ package spark.ml.regression +import scala.util.Random + +import org.jblas.DoubleMatrix + import spark.{RDD, SparkContext} import spark.ml.util.MLUtils -import org.apache.commons.math3.distribution.NormalDistribution -import org.jblas.DoubleMatrix - object RidgeRegressionGenerator { @@ -38,10 +39,9 @@ object RidgeRegressionGenerator { val X = DoubleMatrix.rand(examplesInPartition, nfeatures) val y = X.mmul(w) - val rnd = new NormalDistribution(0, eps) - rnd.reseedRandomGenerator(42 + p) + val rnd = new Random(42 + p) - val normalValues = Array.fill[Double](examplesInPartition)(rnd.sample()) + val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) val yObs = new DoubleMatrix(normalValues).addi(y) Iterator.tabulate(examplesInPartition) { i => diff --git a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index ce388ecc26..53d9789979 100644 --- a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -1,22 +1,23 @@ package spark.ml.regression +import scala.util.Random + +import org.scalatest.FunSuite + import spark.SparkContext import spark.SparkContext._ import spark.Logging -import org.apache.commons.math3.distribution.NormalDistribution -import org.scalatest.FunSuite class LogisticRegressionSuite extends FunSuite with Logging { // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("logistic regression") { val nPoints = 10000 - val rnd = new NormalDistribution(0, 1) - rnd.reseedRandomGenerator(42) + val rnd = new Random(42) val sc = new SparkContext("local", "test") - val x1 = Array.fill[Double](nPoints)(rnd.sample()) + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) val A = 2.0 val B = -1.5 diff --git a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala index 6d5f13d6f5..795cda1379 100644 --- a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -1,25 +1,25 @@ package spark.ml.regression +import scala.util.Random + +import org.scalatest.FunSuite + import spark.SparkContext import spark.SparkContext._ -import org.apache.commons.math3.distribution.NormalDistribution -import org.scalatest.FunSuite class RidgeRegressionSuite extends FunSuite { // Test if we can correctly learn Y = 3 + X1 + X2 when // X1 and X2 are collinear. test("multi-collinear variables") { - val rnd = new NormalDistribution(0, 1) - rnd.reseedRandomGenerator(43) + val rnd = new Random(43) val sc = new SparkContext("local", "test") - val x1 = Array.fill[Double](20)(rnd.sample()) + val x1 = Array.fill[Double](20)(rnd.nextGaussian()) // Pick a mean close to mean of x1 - val rnd1 = new NormalDistribution(0.1, 0.01) - rnd1.reseedRandomGenerator(42) - val x2 = Array.fill[Double](20)(rnd1.sample()) + val rnd1 = new Random(42) //new NormalDistribution(0.1, 0.01) + val x2 = Array.fill[Double](20)(0.1 + rnd1.nextGaussian() * 0.01) val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index aa877ad4a7..5dbb5d4a65 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -224,8 +224,7 @@ object SparkBuild extends Build { def mlSettings = sharedSettings ++ Seq( name := "spark-ml", libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3", - "org.apache.commons" % "commons-math3" % "3.2" + "org.jblas" % "jblas" % "1.2.3" ) ) From d903b3887f13b5ef500cd31227b4563d372e6cd8 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 19:09:17 +0200 Subject: [PATCH 115/419] Initial implementation of Alternating Least Squares. Includes unit tests and sample data to run on. --- ml/data/als/test.data | 16 + ml/src/main/scala/spark/ml/als/ALS.scala | 386 ++++++++++++++++++ .../ml/als/MatrixFactorizationModel.scala | 22 + ml/src/test/scala/spark/ml/als/ALSSuite.scala | 80 ++++ 4 files changed, 504 insertions(+) create mode 100644 ml/data/als/test.data create mode 100644 ml/src/main/scala/spark/ml/als/ALS.scala create mode 100644 ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala create mode 100644 ml/src/test/scala/spark/ml/als/ALSSuite.scala diff --git a/ml/data/als/test.data b/ml/data/als/test.data new file mode 100644 index 0000000000..e476cc23e0 --- /dev/null +++ b/ml/data/als/test.data @@ -0,0 +1,16 @@ +1,1,5.0 +1,2,1.0 +1,3,5.0 +1,4,1.0 +2,1,5.0 +2,2,1.0 +2,3,5.0 +2,4,1.0 +3,1,1.0 +3,2,5.0 +3,3,1.0 +3,4,5.0 +4,1,1.0 +4,2,5.0 +4,3,1.0 +4,4,5.0 diff --git a/ml/src/main/scala/spark/ml/als/ALS.scala b/ml/src/main/scala/spark/ml/als/ALS.scala new file mode 100644 index 0000000000..84339780e2 --- /dev/null +++ b/ml/src/main/scala/spark/ml/als/ALS.scala @@ -0,0 +1,386 @@ +package spark.ml.als + +import scala.collection.mutable.{ArrayBuffer, BitSet} +import scala.util.Random + +import spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import spark.storage.StorageLevel +import spark.SparkContext._ + +import org.jblas.{DoubleMatrix, SimpleBlas, Solve} + + +/** + * Out-link information for a user or product block. This includes the original user/product IDs + * of the elements within this block, and the list of destination blocks that each user or + * product will need to send its feature vector to. + */ +private[als] case class OutLinkBlock( + elementIds: Array[Int], shouldSend: Array[BitSet]) + + +/** + * In-link information for a user (or product) block. This includes the original user/product IDs + * of the elements within this block, as well as an array of indices and ratings that specify + * which user in the block will be rated by which products from each product block (or vice-versa). + * Specifically, if this InLinkBlock is for users, ratingsForBlock(b)(i) will contain two arrays, + * indices and ratings, for the i'th product that will be sent to us by product block b (call this + * P). These arrays represent the users that product P had ratings for (by their index in this + * block), as well as the corresponding rating for each one. We can thus use this information when + * we get product block b's message to update the corresponding users. + */ +private[als] case class InLinkBlock( + elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[Double])]]) + + +/** + * Alternating Least Squares matrix factorization. + * + * This is a blocked implementation of the ALS factorization algorithm that groups the two sets + * of factors (referred to as "users" and "products") into blocks and reduces communication by only + * sending one copy of each user vector to each product block on each iteration, and only for the + * product blocks that need that user's feature vector. This is achieved by precomputing some + * information about the ratings matrix to determine the "out-links" of each user (which blocks of + * products it will contribute to) and "in-link" information for each product (which of the feature + * vectors it receives from each user block it will depend on). This allows us to send only an + * array of feature vectors between each user block and product block, and have the product block + * find the users' ratings and update the products based on these messages. + */ +class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var lambda: Double) + extends Serializable +{ + def this() = this(-1, 10, 10, 0.01) + + /** + * Set the number of blocks to parallelize the computation into; pass -1 for an auto-configured + * number of blocks. + */ + def setBlocks(numBlocks: Int): ALS = { + this.numBlocks = numBlocks + this + } + + /** Set the rank of the feature matrices computed (number of features). */ + def setRank(rank: Int): ALS = { + this.rank = rank + this + } + + /** Set the total number of iterations */ + def setIterations(iterations: Int): ALS = { + this.iterations = iterations + this + } + + /** Set the regularization parameter, lambda */ + def setLambda(lambda: Double): ALS = { + this.lambda = lambda + this + } + + /** + * Run ALS with the configured parmeters on an input RDD of (user, product, rating) triples. + * Returns a MatrixFactorizationModel with feature vectors for each user and product. + */ + def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { + val numBlocks = if (this.numBlocks == -1) { + math.max(ratings.context.defaultParallelism, ratings.partitions.size) + } else { + this.numBlocks + } + + val partitioner = new HashPartitioner(numBlocks) + + val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } + val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } + + val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) + val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) + + // Initialize user and product factors deterministically. + var users = userOutLinks.mapValues(_.elementIds.map(u => makeInitialFactor(rank, u))) + var products = productOutLinks.mapValues(_.elementIds.map(p => makeInitialFactor(rank, -p))) + + for (iter <- 0 until iterations) { + // perform ALS update + products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda) + users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda) + } + + // Flatten and cache the two final RDDs to un-block them + val usersOut = users.join(userOutLinks).flatMap { case (b, (factors, outLinkBlock)) => + for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i)) + } + val productsOut = products.join(productOutLinks).flatMap { case (b, (factors, outLinkBlock)) => + for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i)) + } + + usersOut.persist() + productsOut.persist() + + new MatrixFactorizationModel(rank, usersOut, productsOut) + } + + /** + * Make the out-links table for a block of the users (or products) dataset given the list of + * (user, product, rating) values for the users in that block (or the opposite for products). + */ + private def makeOutLinkBlock(numBlocks: Int, ratings: Array[(Int, Int, Double)]): OutLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) + for ((u, p, r) <- ratings) { + shouldSend(userIdToPos(u))(p % numBlocks) = true + } + OutLinkBlock(userIds, shouldSend) + } + + /** + * Make the in-links table for a block of the users (or products) dataset given a list of + * (user, product, rating) values for the users in that block (or the opposite for products). + */ + private def makeInLinkBlock(numBlocks: Int, ratings: Array[(Int, Int, Double)]): InLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) + for (productBlock <- 0 until numBlocks) { + val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) + val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) + .toArray + .sortBy(_._1) + .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} + ratingsForBlock(productBlock) = ratingsByProduct + } + InLinkBlock(userIds, ratingsForBlock) + } + + /** + * Make RDDs of InLinkBlocks and OutLinkBlocks given an RDD of (blockId, (u, p, r)) values for + * the users (or (blockId, (p, u, r)) for the products). We create these simultaneously to avoid + * having to shuffle the (blockId, (u, p, r)) RDD twice, or to cache it. + */ + private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, (Int, Int, Double))]) + : (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) = + { + val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) + val links = grouped.mapPartitionsWithIndex((blockId, elements) => { + val ratings = elements.map(_._2).toArray + Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) + }, true) + links.persist() + (links.mapValues(_._1), links.mapValues(_._2)) + } + + /** + * Make a random initial factor vector with the given seed. + * TODO: Initialize things using mapPartitionsWithIndex to make it faster? + */ + private def makeInitialFactor(rank: Int, seed: Int): Array[Double] = { + val rand = new Random(seed) + Array.fill(rank)(rand.nextDouble) + } + + /** + * Compute the user feature vectors given the current products (or vice-versa). This first joins + * the products with their out-links to generate a set of messages to each destination block + * (specifically, the features for the products that user block cares about), then groups these + * by destination and joins them with the in-link info to figure out how to update each user. + * It returns an RDD of new feature vectors for each user block. + */ + private def updateFeatures( + products: RDD[(Int, Array[Array[Double]])], + productOutLinks: RDD[(Int, OutLinkBlock)], + userInLinks: RDD[(Int, InLinkBlock)], + partitioner: Partitioner, + rank: Int, + lambda: Double) + : RDD[(Int, Array[Array[Double]])] = + { + val numBlocks = products.partitions.size + productOutLinks.join(products).flatMap { case (bid, (outLinkBlock, factors)) => + val toSend = Array.fill(numBlocks)(new ArrayBuffer[Array[Double]]) + for (p <- 0 until outLinkBlock.elementIds.length; userBlock <- 0 until numBlocks) { + if (outLinkBlock.shouldSend(p)(userBlock)) { + toSend(userBlock) += factors(p) + } + } + toSend.zipWithIndex.map{ case (buf, idx) => (idx, (bid, buf.toArray)) } + }.groupByKey(partitioner) + .join(userInLinks) + .mapValues{ case (messages, inLinkBlock) => updateBlock(messages, inLinkBlock, rank, lambda) } + } + + /** + * Compute the new feature vectors for a block of the users matrix given the list of factors + * it received from each product and its InLinkBlock. + */ + def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, + rank: Int, lambda: Double) + : Array[Array[Double]] = + { + // Sort the incoming block factor messages by block ID and make them an array + val blockFactors = messages.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] + val numBlocks = blockFactors.length + val numUsers = inLinkBlock.elementIds.length + + // We'll sum up the XtXes using vectors that represent only the lower-triangular part, since + // the matrices are symmetric + val triangleSize = rank * (rank + 1) / 2 + val userXtX = Array.fill(numUsers)(DoubleMatrix.zeros(triangleSize)) + val userXy = Array.fill(numUsers)(DoubleMatrix.zeros(rank)) + + // Some temp variables to avoid memory allocation + val tempXtX = DoubleMatrix.zeros(triangleSize) + val fullXtX = DoubleMatrix.zeros(rank, rank) + + // Compute the XtX and Xy values for each user by adding products it rated in each product block + for (productBlock <- 0 until numBlocks) { + for (p <- 0 until blockFactors(productBlock).length) { + val x = new DoubleMatrix(blockFactors(productBlock)(p)) + fillXtX(x, tempXtX) + val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) + for (i <- 0 until us.length) { + userXtX(us(i)).addi(tempXtX) + SimpleBlas.axpy(rs(i), x, userXy(us(i))) + } + } + } + + // Solve the least-squares problem for each user and return the new feature vectors + userXtX.zipWithIndex.map{ case (triangularXtX, index) => + // Compute the full XtX matrix from the lower-triangular part we got above + fillFullMatrix(triangularXtX, fullXtX) + // Add regularization + (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda) + // Solve the resulting matrix, which is symmetric and positive-definite + Solve.solvePositive(fullXtX, userXy(index)).data + } + } + + /** + * Set xtxDest to the lower-triangular part of x transpose * x. For efficiency in summing + * these matrices, we store xtxDest as only rank * (rank+1) / 2 values, namely the values + * at (0,0), (1,0), (1,1), (2,0), (2,1), (2,2), etc in that order. + */ + private def fillXtX(x: DoubleMatrix, xtxDest: DoubleMatrix) { + var i = 0 + var pos = 0 + while (i < x.length) { + var j = 0 + while (j <= i) { + xtxDest.data(pos) = x.data(i) * x.data(j) + pos += 1 + j += 1 + } + i += 1 + } + } + + /** + * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square + * matrix that it represents, storing it into destMatrix. + */ + private def fillFullMatrix(triangularMatrix: DoubleMatrix, destMatrix: DoubleMatrix) { + val rank = destMatrix.rows + var i = 0 + var pos = 0 + while (i < rank) { + var j = 0 + while (j <= i) { + destMatrix.data(i*rank + j) = triangularMatrix.data(pos) + destMatrix.data(j*rank + i) = triangularMatrix.data(pos) + pos += 1 + j += 1 + } + i += 1 + } + } +} + + +/** + * Top-level methods for calling Alternating Least Squares (ALS) matrix factorizaton. + */ +object ALS { + /** + * Train a matrix factorization model given an RDD of ratings given by users to some products, + * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the + * product of two lower-rank matrices of a given rank (number of features). To solve for these + * features, we run a given number of iterations of ALS. This is done using a level of + * parallelism given by `blocks`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + * @param lambda regularization factor (recommended: 0.01) + * @param blocks level of parallelism to split computation into + */ + def train( + ratings: RDD[(Int, Int, Double)], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int) + : MatrixFactorizationModel = + { + new ALS(blocks, rank, iterations, lambda).train(ratings) + } + + /** + * Train a matrix factorization model given an RDD of ratings given by users to some products, + * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the + * product of two lower-rank matrices of a given rank (number of features). To solve for these + * features, we run a given number of iterations of ALS. The level of parallelism is determined + * automatically based on the number of partitions in `ratings`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + * @param lambda regularization factor (recommended: 0.01) + */ + def train(ratings: RDD[(Int, Int, Double)], rank: Int, iterations: Int, lambda: Double) + : MatrixFactorizationModel = + { + train(ratings, rank, iterations, lambda, -1) + } + + /** + * Train a matrix factorization model given an RDD of ratings given by users to some products, + * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the + * product of two lower-rank matrices of a given rank (number of features). To solve for these + * features, we run a given number of iterations of ALS. The level of parallelism is determined + * automatically based on the number of partitions in `ratings`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + */ + def train(ratings: RDD[(Int, Int, Double)], rank: Int, iterations: Int) + : MatrixFactorizationModel = + { + train(ratings, rank, iterations, 0.01, -1) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: ALS ") + System.exit(1) + } + val (master, ratingsFile, rank, iters, outputDir) = + (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) + val sc = new SparkContext(master, "ALS") + val ratings = sc.textFile(ratingsFile).map { line => + val fields = line.split(',') + (fields(0).toInt, fields(1).toInt, fields(2).toDouble) + } + val model = ALS.train(ratings, rank, iters) + model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } + .saveAsTextFile(outputDir + "/userFeatures") + model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } + .saveAsTextFile(outputDir + "/productFeatures") + println("Final user/product features written to " + outputDir) + System.exit(0) + } +} diff --git a/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala b/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala new file mode 100644 index 0000000000..52a8984bc7 --- /dev/null +++ b/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala @@ -0,0 +1,22 @@ +package spark.ml.als + +import spark.RDD +import spark.SparkContext._ + +import org.jblas._ + +class MatrixFactorizationModel( + val rank: Int, + val userFeatures: RDD[(Int, Array[Double])], + val productFeatures: RDD[(Int, Array[Double])]) +{ + /** Predict the rating of one user for one product. */ + def predict(user: Int, product: Int): Double = { + val userVector = new DoubleMatrix(userFeatures.lookup(user).head) + val productVector = new DoubleMatrix(productFeatures.lookup(product).head) + userVector.dot(productVector) + } + + // TODO: Figure out what good bulk prediction methods would look like. + // Probably want a way to get the top users for a product or vice-versa. +} diff --git a/ml/src/test/scala/spark/ml/als/ALSSuite.scala b/ml/src/test/scala/spark/ml/als/ALSSuite.scala new file mode 100644 index 0000000000..70f350857f --- /dev/null +++ b/ml/src/test/scala/spark/ml/als/ALSSuite.scala @@ -0,0 +1,80 @@ +package spark.ml.als + +import scala.util.Random + +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ +import spark.Logging + +import org.jblas._ + + +class ALSSuite extends FunSuite with Logging { + + test("rank-1 matrices") { + testALS(10, 20, 1, 5, 0.5, 0.1) + } + + test("rank-2 matrices") { + testALS(20, 30, 2, 10, 0.7, 0.3) + } + + /** + * Test if we can correctly factorize R = U * P where U and P are of known rank. + * + * @param users number of users + * @param products number of products + * @param features number of features (rank of problem) + * @param iterations number of iterations to run + * @param samplingRate what fraction of the user-product pairs are known + * @param matchThreshold max difference allowed to consider a predicted rating correct + */ + def testALS(users: Int, products: Int, features: Int, iterations: Int, + samplingRate: Double, matchThreshold: Double) + { + val rand = new Random(42) + + // Create a random matrix with uniform values from -1 to 1 + def randomMatrix(m: Int, n: Int) = + new DoubleMatrix(m, n, Array.fill(m * n)(rand.nextDouble() * 2 - 1): _*) + + val userMatrix = randomMatrix(users, features) + val productMatrix = randomMatrix(features, products) + val trueRatings = userMatrix.mmul(productMatrix) + + val sampledRatings = { + for (u <- 0 until users; p <- 0 until products if rand.nextDouble() < samplingRate) + yield (u, p, trueRatings.get(u, p)) + } + + val sc = new SparkContext("local", "test") + + val model = ALS.train(sc.parallelize(sampledRatings), features, iterations) + + val predictedU = new DoubleMatrix(users, features) + for ((u, vec) <- model.userFeatures.collect(); i <- 0 until features) { + predictedU.put(u, i, vec(i)) + } + val predictedP = new DoubleMatrix(products, features) + for ((p, vec) <- model.productFeatures.collect(); i <- 0 until features) { + predictedP.put(p, i, vec(i)) + } + val predictedRatings = predictedU.mmul(predictedP.transpose) + + for (u <- 0 until users; p <- 0 until products) { + val prediction = predictedRatings.get(u, p) + val correct = trueRatings.get(u, p) + if (math.abs(prediction - correct) > matchThreshold) { + fail("Model failed to predict (%d, %d): %f vs %f\ncorr: %s\npred: %s\nU: %s\n P: %s".format( + u, p, correct, prediction, trueRatings, predictedRatings, predictedU, predictedP)) + } + } + + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } +} + From 6f0ebb2db24272390ebd481e50a08336e37d2b1d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 19:13:22 +0200 Subject: [PATCH 116/419] Remove unused import --- ml/src/main/scala/spark/ml/als/ALS.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/ml/src/main/scala/spark/ml/als/ALS.scala b/ml/src/main/scala/spark/ml/als/ALS.scala index 84339780e2..dca5bfc8f1 100644 --- a/ml/src/main/scala/spark/ml/als/ALS.scala +++ b/ml/src/main/scala/spark/ml/als/ALS.scala @@ -4,7 +4,6 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random import spark.{HashPartitioner, Partitioner, SparkContext, RDD} -import spark.storage.StorageLevel import spark.SparkContext._ import org.jblas.{DoubleMatrix, SimpleBlas, Solve} From 3c046a6eca182d720b58f5cdfd17b6201664e716 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 19:24:06 +0200 Subject: [PATCH 117/419] Some small fixes to ALS. --- ml/src/main/scala/spark/ml/als/ALS.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/ml/src/main/scala/spark/ml/als/ALS.scala b/ml/src/main/scala/spark/ml/als/ALS.scala index dca5bfc8f1..a1ae28a38d 100644 --- a/ml/src/main/scala/spark/ml/als/ALS.scala +++ b/ml/src/main/scala/spark/ml/als/ALS.scala @@ -4,6 +4,7 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random import spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import spark.storage.StorageLevel import spark.SparkContext._ import org.jblas.{DoubleMatrix, SimpleBlas, Solve} @@ -52,26 +53,26 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l /** * Set the number of blocks to parallelize the computation into; pass -1 for an auto-configured - * number of blocks. + * number of blocks. Default: -1. */ def setBlocks(numBlocks: Int): ALS = { this.numBlocks = numBlocks this } - /** Set the rank of the feature matrices computed (number of features). */ + /** Set the rank of the feature matrices computed (number of features). Default: 10. */ def setRank(rank: Int): ALS = { this.rank = rank this } - /** Set the total number of iterations */ + /** Set the number of iterations to run. Default: 10. */ def setIterations(iterations: Int): ALS = { this.iterations = iterations this } - /** Set the regularization parameter, lambda */ + /** Set the regularization parameter, lambda. Default: 0.01. */ def setLambda(lambda: Double): ALS = { this.lambda = lambda this @@ -168,7 +169,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val ratings = elements.map(_._2).toArray Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) }, true) - links.persist() + links.persist(StorageLevel.MEMORY_AND_DISK) (links.mapValues(_._1), links.mapValues(_._2)) } From d3ce898b8eec066d69892d29b0c6531790902d78 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 20:00:56 +0200 Subject: [PATCH 118/419] Scaffolding and model for K-means --- .../scala/spark/ml/clustering/KMeans.scala | 117 ++++++++++++++++++ .../spark/ml/clustering/KMeansModel.scala | 25 ++++ ml/src/main/scala/spark/ml/util/MLUtils.scala | 15 +++ 3 files changed, 157 insertions(+) create mode 100644 ml/src/main/scala/spark/ml/clustering/KMeans.scala create mode 100644 ml/src/main/scala/spark/ml/clustering/KMeansModel.scala diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/ml/src/main/scala/spark/ml/clustering/KMeans.scala new file mode 100644 index 0000000000..fd15da9b43 --- /dev/null +++ b/ml/src/main/scala/spark/ml/clustering/KMeans.scala @@ -0,0 +1,117 @@ +package spark.ml.clustering + +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +import spark.{SparkContext, RDD} +import spark.SparkContext._ +import spark.Logging + +import org.jblas.DoubleMatrix + + +/** + * K-means clustering with support for multiple parallel runs and a k-means++ like initialization + * mode (the k-means|| algorithm by TODO). When multiple concurrent runs are requested, they are + * executed together with joint passes over the data for efficiency. + * + * This is an iterative algorithm that will make multiple passes over the data, so any RDDs given + * to it should be cached by the user. + */ +class KMeans private ( + var k: Int, + var maxIterations: Int, + var runs: Int, + var initializationMode: String, + var initializationSteps: Int) + extends Serializable with Logging +{ + // Initialization mode names + private val RANDOM = "random" + private val K_MEANS_PARALLEL = "k-means||" + + def this() = this(2, 20, 1, "k-means||", 5) + + /** Set the number of clusters to create (k). Default: 2. */ + def setK(k: Int): KMeans = { + this.k = k + this + } + + /** Set maximum number of iterations to run. Default: 20. */ + def setMaxIterations(maxIterations: Int): KMeans = { + this.maxIterations = maxIterations + this + } + + /** + * Set the initialization algorithm. This can be either "random" to choose random points as + * initial cluster centers, or "k-means||" to use a parallel version of k-means++ discussed + * in (TODO). Default: k-means||. + */ + def setInitializationMode(initializationMode: String): KMeans = { + if (initializationMode != RANDOM && initializationMode != K_MEANS_PARALLEL) { + throw new IllegalArgumentException("Invalid initialization mode: " + initializationMode) + } + this.initializationMode = initializationMode + this + } + + /** + * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm + * this many times with random starting conditions (configured by the initialization mode), then + * return the best clustering found over any run. Default: 1. + */ + def setRuns(runs: Int): KMeans = { + this.runs = runs + this + } + + /** + * Set the number of steps for the k-means|| initialization mode. This is an advanced + * setting -- the default of 5 is almost always enough. Default: 5. + */ + def setInitializationSteps(initializationSteps: Int): KMeans = { + this.initializationSteps = initializationSteps + this + } + + /** + * Train a K-means model on the given set of points; `data` should be cached for high + * performance, because this is an iterative algorithm. + */ + def train(data: RDD[Array[Double]]): KMeansModel = { + // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable + + null + } +} + + +/** + * Top-level methods for calling K-means clustering. + */ +object KMeans { + def train( + data: RDD[Array[Double]], + k: Int, + maxIterations: Int, + runs: Int, + initializationMode: String) + : KMeansModel = + { + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .setRuns(runs) + .setInitializationMode(initializationMode) + .train(data) + } + + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { + train(data, k, maxIterations, runs, "k-means||") + } + + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int): KMeansModel = { + train(data, k, maxIterations, 1, "k-means||") + } +} diff --git a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala new file mode 100644 index 0000000000..f2f26c47b8 --- /dev/null +++ b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala @@ -0,0 +1,25 @@ +package spark.ml.clustering + +import spark.ml.util.MLUtils + +/** + * A clustering model for K-means. Each point belongs to the cluster with the closest center. + */ +class KMeansModel(val clusterCenters: Array[Array[Double]]) { + /** Total number of clusters. */ + def k: Int = clusterCenters.length + + /** Return the cluster index that a given point belongs to. */ + def predict(point: Array[Double]): Int = { + var bestDist = Double.PositiveInfinity + var bestIndex = -1 + for (i <- 0 until k) { + val dist = MLUtils.squaredDistance(clusterCenters(i), point) + if (dist < bestDist) { + bestDist = dist + bestIndex = i + } + } + bestIndex + } +} diff --git a/ml/src/main/scala/spark/ml/util/MLUtils.scala b/ml/src/main/scala/spark/ml/util/MLUtils.scala index 76e1bbd36d..48292649c3 100644 --- a/ml/src/main/scala/spark/ml/util/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/util/MLUtils.scala @@ -76,4 +76,19 @@ object MLUtils { (yMean, xColMean, xColSd) } + + /** + * Return the squared Euclidean distance between two vectors. + */ + def squaredDistance(v1: Array[Double], v2: Array[Double]): Double = { + if (v1.length != v2.length) { + throw new IllegalArgumentException("Vector sizes don't match") + } + var i = 0 + var sum = 0.0 + while (i < v1.length) { + sum += (v1(i) - v2(i)) * (v1(i) - v2(i)) + } + sum + } } From 43dae967d7095d4ebd1ab5919e8907da640e48d3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 15:16:24 -0400 Subject: [PATCH 119/419] Renamed "als" package to "recommendation" --- .../spark/ml/{als => recommendation}/ALS.scala | 14 +++++++------- .../MatrixFactorizationModel.scala | 2 +- .../ml/{als => recommendation}/ALSSuite.scala | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) rename ml/src/main/scala/spark/ml/{als => recommendation}/ALS.scala (98%) rename ml/src/main/scala/spark/ml/{als => recommendation}/MatrixFactorizationModel.scala (95%) rename ml/src/test/scala/spark/ml/{als => recommendation}/ALSSuite.scala (98%) diff --git a/ml/src/main/scala/spark/ml/als/ALS.scala b/ml/src/main/scala/spark/ml/recommendation/ALS.scala similarity index 98% rename from ml/src/main/scala/spark/ml/als/ALS.scala rename to ml/src/main/scala/spark/ml/recommendation/ALS.scala index a1ae28a38d..6717315fa7 100644 --- a/ml/src/main/scala/spark/ml/als/ALS.scala +++ b/ml/src/main/scala/spark/ml/recommendation/ALS.scala @@ -1,4 +1,4 @@ -package spark.ml.als +package spark.ml.recommendation import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random @@ -15,7 +15,7 @@ import org.jblas.{DoubleMatrix, SimpleBlas, Solve} * of the elements within this block, and the list of destination blocks that each user or * product will need to send its feature vector to. */ -private[als] case class OutLinkBlock( +private[recommendation] case class OutLinkBlock( elementIds: Array[Int], shouldSend: Array[BitSet]) @@ -29,7 +29,7 @@ private[als] case class OutLinkBlock( * block), as well as the corresponding rating for each one. We can thus use this information when * we get product block b's message to update the corresponding users. */ -private[als] case class InLinkBlock( +private[recommendation] case class InLinkBlock( elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[Double])]]) @@ -51,7 +51,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l { def this() = this(-1, 10, 10, 0.01) - /** + /** * Set the number of blocks to parallelize the computation into; pass -1 for an auto-configured * number of blocks. Default: -1. */ @@ -198,7 +198,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l lambda: Double) : RDD[(Int, Array[Array[Double]])] = { - val numBlocks = products.partitions.size + val numBlocks = products.partitions.size productOutLinks.join(products).flatMap { case (bid, (outLinkBlock, factors)) => val toSend = Array.fill(numBlocks)(new ArrayBuffer[Array[Double]]) for (p <- 0 until outLinkBlock.elementIds.length; userBlock <- 0 until numBlocks) { @@ -224,7 +224,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val blockFactors = messages.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] val numBlocks = blockFactors.length val numUsers = inLinkBlock.elementIds.length - + // We'll sum up the XtXes using vectors that represent only the lower-triangular part, since // the matrices are symmetric val triangleSize = rank * (rank + 1) / 2 @@ -321,7 +321,7 @@ object ALS { ratings: RDD[(Int, Int, Double)], rank: Int, iterations: Int, - lambda: Double, + lambda: Double, blocks: Int) : MatrixFactorizationModel = { diff --git a/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala similarity index 95% rename from ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala rename to ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala index 52a8984bc7..7bd33a87ee 100644 --- a/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala +++ b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala @@ -1,4 +1,4 @@ -package spark.ml.als +package spark.ml.recommendation import spark.RDD import spark.SparkContext._ diff --git a/ml/src/test/scala/spark/ml/als/ALSSuite.scala b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/als/ALSSuite.scala rename to ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala index 70f350857f..957ab51feb 100644 --- a/ml/src/test/scala/spark/ml/als/ALSSuite.scala +++ b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.als +package spark.ml.recommendation import scala.util.Random From 6586c5e28b4ccbb569ca8cf149bd181d0d8e1e33 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 16:52:51 -0400 Subject: [PATCH 120/419] Added a SparkContext accessor to RDD --- core/src/main/scala/spark/RDD.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index f336c2ea1e..8e1e18c06c 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -105,6 +105,9 @@ abstract class RDD[T: ClassManifest]( // Methods and fields available on all RDDs // ======================================================================= + /** The SparkContext that created this RDD. */ + def sparkContext: SparkContext = sc + /** A unique ID for this RDD (within its SparkContext). */ val id: Int = sc.newRddId() From 39684eafe3dbfdc0563c95c05e69350fe39c71f9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 18:08:52 -0400 Subject: [PATCH 121/419] Formatting --- ml/src/main/scala/spark/ml/recommendation/ALS.scala | 2 +- .../spark/ml/recommendation/MatrixFactorizationModel.scala | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/ml/src/main/scala/spark/ml/recommendation/ALS.scala b/ml/src/main/scala/spark/ml/recommendation/ALS.scala index 6717315fa7..abf54216a5 100644 --- a/ml/src/main/scala/spark/ml/recommendation/ALS.scala +++ b/ml/src/main/scala/spark/ml/recommendation/ALS.scala @@ -365,7 +365,7 @@ object ALS { def main(args: Array[String]) { if (args.length != 5) { - println("Usage: ALS ") + println("Usage: ALS ") System.exit(1) } val (master, ratingsFile, rank, iters, outputDir) = diff --git a/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala index 7bd33a87ee..cfdf2ba523 100644 --- a/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala +++ b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala @@ -6,9 +6,10 @@ import spark.SparkContext._ import org.jblas._ class MatrixFactorizationModel( - val rank: Int, - val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) + val rank: Int, + val userFeatures: RDD[(Int, Array[Double])], + val productFeatures: RDD[(Int, Array[Double])]) + extends Serializable { /** Predict the rating of one user for one product. */ def predict(user: Int, product: Int): Double = { From 52f491125ecc26821842fe88786b4b0e7c01499b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 18:09:19 -0400 Subject: [PATCH 122/419] Implementation of k-means and k-means|| --- .../scala/spark/ml/clustering/KMeans.scala | 230 ++++++++++++++++-- .../spark/ml/clustering/KMeansModel.scala | 13 +- .../spark/ml/clustering/LocalKMeans.scala | 88 +++++++ ml/src/main/scala/spark/ml/util/MLUtils.scala | 1 + 4 files changed, 316 insertions(+), 16 deletions(-) create mode 100644 ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/ml/src/main/scala/spark/ml/clustering/KMeans.scala index fd15da9b43..d1786b0812 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeans.scala @@ -6,14 +6,15 @@ import scala.util.Random import spark.{SparkContext, RDD} import spark.SparkContext._ import spark.Logging +import spark.ml.util.MLUtils import org.jblas.DoubleMatrix /** * K-means clustering with support for multiple parallel runs and a k-means++ like initialization - * mode (the k-means|| algorithm by TODO). When multiple concurrent runs are requested, they are - * executed together with joint passes over the data for efficiency. + * mode (the k-means|| algorithm by Bahmani et al). When multiple concurrent runs are requested, + * they are executed together with joint passes over the data for efficiency. * * This is an iterative algorithm that will make multiple passes over the data, so any RDDs given * to it should be cached by the user. @@ -23,14 +24,13 @@ class KMeans private ( var maxIterations: Int, var runs: Int, var initializationMode: String, - var initializationSteps: Int) + var initializationSteps: Int, + var epsilon: Double) extends Serializable with Logging { - // Initialization mode names - private val RANDOM = "random" - private val K_MEANS_PARALLEL = "k-means||" + private type ClusterCenters = Array[Array[Double]] - def this() = this(2, 20, 1, "k-means||", 5) + def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) /** Set the number of clusters to create (k). Default: 2. */ def setK(k: Int): KMeans = { @@ -46,11 +46,11 @@ class KMeans private ( /** * Set the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel version of k-means++ discussed - * in (TODO). Default: k-means||. + * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ + * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. */ def setInitializationMode(initializationMode: String): KMeans = { - if (initializationMode != RANDOM && initializationMode != K_MEANS_PARALLEL) { + if (initializationMode != KMeans.RANDOM && initializationMode != KMeans.K_MEANS_PARALLEL) { throw new IllegalArgumentException("Invalid initialization mode: " + initializationMode) } this.initializationMode = initializationMode @@ -63,6 +63,9 @@ class KMeans private ( * return the best clustering found over any run. Default: 1. */ def setRuns(runs: Int): KMeans = { + if (runs <= 0) { + throw new IllegalArgumentException("Number of runs must be positive") + } this.runs = runs this } @@ -72,10 +75,22 @@ class KMeans private ( * setting -- the default of 5 is almost always enough. Default: 5. */ def setInitializationSteps(initializationSteps: Int): KMeans = { + if (initializationSteps <= 0) { + throw new IllegalArgumentException("Number of initialization steps must be positive") + } this.initializationSteps = initializationSteps this } + /** + * Set the distance threshold within which we've consider centers to have converged. + * If all centers move less than this Euclidean distance, we stop iterating one run. + */ + def setEpsilon(epsilon: Double): KMeans = { + this.epsilon = epsilon + this + } + /** * Train a K-means model on the given set of points; `data` should be cached for high * performance, because this is an iterative algorithm. @@ -83,7 +98,139 @@ class KMeans private ( def train(data: RDD[Array[Double]]): KMeansModel = { // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable - null + val sc = data.sparkContext + + var centers = if (initializationMode == KMeans.RANDOM) { + initRandom(data) + } else { + initKMeansParallel(data) + } + + val active = Array.fill(runs)(true) + val costs = Array.fill(runs)(0.0) + + var activeRuns = new ArrayBuffer[Int] ++ (0 until runs) + var iteration = 0 + + // Execute iterations of Lloyd's algorithm until all runs have converged + while (iteration < maxIterations && !activeRuns.isEmpty) { + type WeightedPoint = (DoubleMatrix, Long) + def mergeContribs(p1: WeightedPoint, p2: WeightedPoint): WeightedPoint = { + (p1._1.addi(p2._1), p1._2 + p2._2) + } + + val activeCenters = activeRuns.map(r => centers(r)).toArray + val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + + // Find the sum and count of points mapping to each center + val totalContribs = data.mapPartitions { points => + val runs = activeCenters.length + val k = activeCenters(0).length + val dims = activeCenters(0)(0).length + + val sums = Array.fill(runs, k)(new DoubleMatrix(dims)) + val counts = Array.fill(runs, k)(0L) + + for (point <- points) { + for ((centers, runIndex) <- activeCenters.zipWithIndex) { + val (bestCenter, cost) = KMeans.findClosest(centers, point) + costAccums(runIndex) += cost + sums(runIndex)(bestCenter).addi(new DoubleMatrix(point)) + counts(runIndex)(bestCenter) += 1 + } + } + + val contribs = for (i <- 0 until runs; j <- 0 until k) yield { + ((i, j), (sums(i)(j), counts(i)(j))) + } + contribs.iterator + }.reduceByKey(mergeContribs).collectAsMap() + + // Update the cluster centers and costs for each active run + for ((run, i) <- activeRuns.zipWithIndex) { + var changed = false + for (j <- 0 until k) { + val (sum, count) = totalContribs((i, j)) + if (count != 0) { + val newCenter = sum.divi(count).data + if (MLUtils.squaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { + changed = true + } + centers(run)(j) = newCenter + } + } + if (!changed) { + active(run) = false + logInfo("Run " + run + " finished in " + (iteration + 1) + " iterations") + } + costs(run) = costAccums(i).value + } + + activeRuns = activeRuns.filter(active(_)) + iteration += 1 + } + + val bestRun = costs.zipWithIndex.min._2 + new KMeansModel(centers(bestRun)) + } + + /** + * Initialize `runs` sets of cluster centers at random. + */ + private def initRandom(data: RDD[Array[Double]]): Array[ClusterCenters] = { + // Sample all the cluster centers in one pass to avoid repeated scans + val sample = data.takeSample(true, runs * k, 1) + Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k)) + } + + /** + * Initialize `runs` sets of cluster centers using the k-means|| algorithm by Bahmani et al. + * (Bahmani et al., Scalable K-Means++, VLDB 2012). This is a variant of k-means++ that tries + * to find with dissimilar cluster centers by starting with a random center and then doing + * passes where more centers are chosen with probability proportional to their squared distance + * to the current cluster set. It results in a provable approximation to an optimal clustering. + * + * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. + */ + private def initKMeansParallel(data: RDD[Array[Double]]): Array[ClusterCenters] = { + // Initialize each run's center to a random point + val sample = data.takeSample(true, runs, 1) + val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r))) + + // On each step, sample 2 * k points on average for each run with probability proportional + // to their squared distance from that run's current centers + for (step <- 0 until initializationSteps) { + val centerArrays = centers.map(_.toArray) + val sumCosts = data.flatMap { point => + for (r <- 0 until runs) yield (r, KMeans.pointCost(centerArrays(r), point)) + }.reduceByKey(_ + _).collectAsMap() + val chosen = data.mapPartitionsWithIndex { (index, points) => + val rand = new Random((step << 16) ^ index) + for { + p <- points + r <- 0 until runs + if rand.nextDouble() < KMeans.pointCost(centerArrays(r), p) * 2 * k / sumCosts(r) + } yield (r, p) + }.collect() + for ((r, p) <- chosen) { + centers(r) += p + } + } + + // Finally, we might have a set of more than k candidate centers for each run; weigh each + // candidate by the number of points in the dataset mapping to it and run a local k-means++ + // on the weighted centers to pick just k of them + val centerArrays = centers.map(_.toArray) + val weightMap = data.flatMap { p => + for (r <- 0 until runs) yield ((r, KMeans.findClosest(centerArrays(r), p)._1), 1.0) + }.reduceByKey(_ + _).collectAsMap() + val finalCenters = (0 until runs).map { r => + val myCenters = centers(r).toArray + val myWeights = (0 until myCenters.length).map(i => weightMap((r, i))).toArray + LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) + } + + finalCenters.toArray } } @@ -92,6 +239,10 @@ class KMeans private ( * Top-level methods for calling K-means clustering. */ object KMeans { + // Initialization mode names + val RANDOM = "random" + val K_MEANS_PARALLEL = "k-means||" + def train( data: RDD[Array[Double]], k: Int, @@ -106,12 +257,61 @@ object KMeans { .setInitializationMode(initializationMode) .train(data) } - + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { - train(data, k, maxIterations, runs, "k-means||") + train(data, k, maxIterations, runs, K_MEANS_PARALLEL) } - + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int): KMeansModel = { - train(data, k, maxIterations, 1, "k-means||") + train(data, k, maxIterations, 1, K_MEANS_PARALLEL) + } + + /** + * Return the index of the closest point in `centers` to `point`, as well as its distance. + */ + private[ml] def findClosest(centers: Array[Array[Double]], point: Array[Double]): (Int, Double) = + { + var bestDistance = Double.PositiveInfinity + var bestIndex = 0 + for (i <- 0 until centers.length) { + val distance = MLUtils.squaredDistance(point, centers(i)) + if (distance < bestDistance) { + bestDistance = distance + bestIndex = i + } + } + (bestIndex, bestDistance) + } + + /** + * Return the K-means cost of a given point against the given cluster centers. + */ + private[ml] def pointCost(centers: Array[Array[Double]], point: Array[Double]): Double = { + var bestDistance = Double.PositiveInfinity + for (i <- 0 until centers.length) { + val distance = MLUtils.squaredDistance(point, centers(i)) + if (distance < bestDistance) { + bestDistance = distance + } + } + bestDistance + } + + def main(args: Array[String]) { + if (args.length != 4) { + println("Usage: KMeans ") + System.exit(1) + } + val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) + val sc = new SparkContext(master, "KMeans") + val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) + val model = KMeans.train(data, k, iters) + val cost = model.computeCost(data) + println("Cluster centers:") + for (c <- model.clusterCenters) { + println(" " + c.mkString(" ")) + } + println("Cost: " + cost) + System.exit(0) } } diff --git a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala index f2f26c47b8..84bfd0f99a 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala @@ -1,11 +1,14 @@ package spark.ml.clustering +import spark.RDD +import spark.SparkContext._ import spark.ml.util.MLUtils + /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel(val clusterCenters: Array[Array[Double]]) { +class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable { /** Total number of clusters. */ def k: Int = clusterCenters.length @@ -22,4 +25,12 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) { } bestIndex } + + /** + * Return the K-means cost (sum of squared distances of points to their nearest center) for this + * model on the given data. + */ + def computeCost(data: RDD[Array[Double]]): Double = { + data.map(p => KMeans.pointCost(clusterCenters, p)).sum + } } diff --git a/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala b/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala new file mode 100644 index 0000000000..03129ef552 --- /dev/null +++ b/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala @@ -0,0 +1,88 @@ +package spark.ml.clustering + +import scala.util.Random + +import org.jblas.{DoubleMatrix, SimpleBlas} + +/** + * An utility object to run K-means locally. This is private to the ML package because it's used + * in the initialization of KMeans but not meant to be publicly exposed. + */ +private[ml] object LocalKMeans { + /** + * Run K-means++ on the weighted point set `points`. This first does the K-means++ + * initialization procedure and then roudns of Lloyd's algorithm. + */ + def kMeansPlusPlus( + seed: Int, + points: Array[Array[Double]], + weights: Array[Double], + k: Int, + maxIterations: Int) + : Array[Array[Double]] = + { + val rand = new Random(seed) + val dimensions = points(0).length + val centers = new Array[Array[Double]](k) + + // Initialize centers by sampling using the k-means++ procedure + centers(0) = pickWeighted(rand, points, weights) + for (i <- 1 until k) { + // Pick the next center with a probability proportional to cost under current centers + val curCenters = centers.slice(0, i) + val sum = points.zip(weights).map { case (p, w) => + w * KMeans.pointCost(curCenters, p) + }.sum + val r = rand.nextDouble() * sum + var cumulativeScore = 0.0 + var j = 0 + while (j < points.length && cumulativeScore < r) { + cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) + j += 1 + } + centers(i) = points(j-1) + } + + // Run up to maxIterations iterations of Lloyd's algorithm + val oldClosest = Array.fill(points.length)(-1) + var iteration = 0 + var moved = true + while (moved && iteration < maxIterations) { + moved = false + val sums = Array.fill(k)(new DoubleMatrix(dimensions)) + val counts = Array.fill(k)(0.0) + for ((p, i) <- points.zipWithIndex) { + val index = KMeans.findClosest(centers, p)._1 + SimpleBlas.axpy(weights(i), new DoubleMatrix(p), sums(index)) + counts(index) += weights(i) + if (index != oldClosest(i)) { + moved = true + oldClosest(i) = index + } + } + // Update centers + for (i <- 0 until k) { + if (counts(i) == 0.0) { + // Assign center to a random point + centers(i) = points(rand.nextInt(points.length)) + } else { + centers(i) = sums(i).divi(counts(i)).data + } + } + iteration += 1 + } + + centers + } + + private def pickWeighted[T](rand: Random, data: Array[T], weights: Array[Double]): T = { + val r = rand.nextDouble() * weights.sum + var i = 0 + var curWeight = 0.0 + while (i < data.length && curWeight < r) { + curWeight += weights(i) + i += 1 + } + data(i - 1) + } +} diff --git a/ml/src/main/scala/spark/ml/util/MLUtils.scala b/ml/src/main/scala/spark/ml/util/MLUtils.scala index 48292649c3..6efa7c81ad 100644 --- a/ml/src/main/scala/spark/ml/util/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/util/MLUtils.scala @@ -88,6 +88,7 @@ object MLUtils { var sum = 0.0 while (i < v1.length) { sum += (v1(i) - v2(i)) * (v1(i) - v2(i)) + i += 1 } sum } From 496c7548bb3af6581f9bdb53cfa5be600d5849fb Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 22 Jun 2013 13:51:30 -0700 Subject: [PATCH 123/419] Change test to use fewer iterations --- .../scala/spark/ml/regression/LogisticRegressionSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index 53d9789979..6216523a9e 100644 --- a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -40,8 +40,8 @@ class LogisticRegressionSuite extends FunSuite with Logging { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(1.0) - .setNumIterations(1000) + val lr = new LogisticRegression().setStepSize(5.0) + .setNumIterations(20) val model = lr.train(testRDD) From cffe3340c5346ab7c21d82cd9c456e9ad17e2e32 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 23:17:15 -0400 Subject: [PATCH 124/419] Fix logistic regression test failure and test suite cleanup --- .../spark/ml/recommendation/ALSSuite.scala | 16 +++++------ .../regression/LogisticRegressionSuite.scala | 28 ++++++++++--------- .../ml/regression/RidgeRegressionSuite.scala | 14 ++++++---- 3 files changed, 31 insertions(+), 27 deletions(-) diff --git a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala index 957ab51feb..c450d5315c 100644 --- a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala +++ b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -2,16 +2,22 @@ package spark.ml.recommendation import scala.util.Random +import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext import spark.SparkContext._ -import spark.Logging import org.jblas._ -class ALSSuite extends FunSuite with Logging { +class ALSSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } test("rank-1 matrices") { testALS(10, 20, 1, 5, 0.5, 0.1) @@ -49,8 +55,6 @@ class ALSSuite extends FunSuite with Logging { yield (u, p, trueRatings.get(u, p)) } - val sc = new SparkContext("local", "test") - val model = ALS.train(sc.parallelize(sampledRatings), features, iterations) val predictedU = new DoubleMatrix(users, features) @@ -71,10 +75,6 @@ class ALSSuite extends FunSuite with Logging { u, p, correct, prediction, trueRatings, predictedRatings, predictedU, predictedP)) } } - - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") } } diff --git a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index 6216523a9e..55f2c5c18e 100644 --- a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -2,21 +2,26 @@ package spark.ml.regression import scala.util.Random +import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext import spark.SparkContext._ -import spark.Logging -class LogisticRegressionSuite extends FunSuite with Logging { +class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("logistic regression") { val nPoints = 10000 val rnd = new Random(42) - val sc = new SparkContext("local", "test") val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) val A = 2.0 @@ -24,14 +29,14 @@ class LogisticRegressionSuite extends FunSuite with Logging { // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) val unifRand = new scala.util.Random(45) - val rLogis = (0 until nPoints).map { i => + val rLogis = (0 until nPoints).map { i => val u = unifRand.nextDouble() math.log(u) - math.log(1.0-u) } - + // y <- A + B*x + rlogis(100) // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => + val y = (0 until nPoints).map { i => val yVal = A + B * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } @@ -40,16 +45,13 @@ class LogisticRegressionSuite extends FunSuite with Logging { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(5.0) + val lr = new LogisticRegression().setStepSize(10.0) .setNumIterations(20) val model = lr.train(testRDD) - assert(model.weights.get(0) >= -1.60 && model.weights.get(0) <= -1.40) - assert(model.intercept >= 1.9 && model.intercept <= 2.1) - - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") + val weight0 = model.weights.get(0) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") } } diff --git a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala index 795cda1379..aed5cbec24 100644 --- a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -2,19 +2,25 @@ package spark.ml.regression import scala.util.Random +import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext import spark.SparkContext._ -class RidgeRegressionSuite extends FunSuite { +class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } // Test if we can correctly learn Y = 3 + X1 + X2 when // X1 and X2 are collinear. test("multi-collinear variables") { val rnd = new Random(43) - val sc = new SparkContext("local", "test") val x1 = Array.fill[Double](20)(rnd.nextGaussian()) // Pick a mean close to mean of x1 @@ -37,9 +43,5 @@ class RidgeRegressionSuite extends FunSuite { assert(model.weights.length === 2) assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) - - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") } } From 652ea0f1d86e77503942a7846f89236b15da2ce9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:15:13 -0700 Subject: [PATCH 125/419] Allow RDD.takeSample to give samples bigger than the RDD Before, when withReplacement was set to true, we would not get a sample bigger than the RDD's count(). Conflicts: core/src/main/scala/spark/RDD.scala core/src/test/scala/spark/RDDSuite.scala --- core/src/main/scala/spark/RDD.scala | 44 +++++++++++++----------- core/src/test/scala/spark/RDDSuite.scala | 34 ++++++++++++++++++ 2 files changed, 58 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 8e1e18c06c..106fb2960f 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -122,7 +122,7 @@ abstract class RDD[T: ClassManifest]( /** User-defined generator of this RDD*/ var generator = Utils.getCallSiteInfo.firstUserClass - + /** Reset generator*/ def setGenerator(_generator: String) = { generator = _generator @@ -284,31 +284,35 @@ abstract class RDD[T: ClassManifest]( def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { var fraction = 0.0 var total = 0 - val multiplier = 3.0 - val initialCount = count() + var multiplier = 3.0 + var initialCount = this.count() var maxSelected = 0 + if (num < 0) { + throw new IllegalArgumentException("Negative number of elements requested") + } + if (initialCount > Integer.MAX_VALUE - 1) { maxSelected = Integer.MAX_VALUE - 1 } else { maxSelected = initialCount.toInt } - if (num > initialCount) { + if (num > initialCount && !withReplacement) { total = maxSelected - fraction = math.min(multiplier * (maxSelected + 1) / initialCount, 1.0) - } else if (num < 0) { - throw(new IllegalArgumentException("Negative number of elements requested")) + fraction = multiplier * (maxSelected + 1) / initialCount } else { - fraction = math.min(multiplier * (num + 1) / initialCount, 1.0) + fraction = multiplier * (num + 1) / initialCount total = num } val rand = new Random(seed) - var samples = this.sample(withReplacement, fraction, rand.nextInt).collect() + var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() + // If the first sample didn't turn out large enough, keep trying to take samples; + // this shouldn't happen often because we use a big multiplier for thei initial size while (samples.length < total) { - samples = this.sample(withReplacement, fraction, rand.nextInt).collect() + samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() } Utils.randomizeInPlace(samples, rand).take(total) @@ -366,7 +370,7 @@ abstract class RDD[T: ClassManifest]( /** * Return an RDD created by piping elements to a forked external process. */ - def pipe(command: String, env: Map[String, String]): RDD[String] = + def pipe(command: String, env: Map[String, String]): RDD[String] = new PipedRDD(this, command, env) @@ -377,24 +381,24 @@ abstract class RDD[T: ClassManifest]( * @param command command to run in forked process. * @param env environment variables to set. * @param printPipeContext Before piping elements, this function is called as an oppotunity - * to pipe context data. Print line function (like out.println) will be + * to pipe context data. Print line function (like out.println) will be * passed as printPipeContext's parameter. - * @param printRDDElement Use this function to customize how to pipe elements. This function - * will be called with each RDD element as the 1st parameter, and the + * @param printRDDElement Use this function to customize how to pipe elements. This function + * will be called with each RDD element as the 1st parameter, and the * print line function (like out.println()) as the 2nd parameter. * An example of pipe the RDD data of groupBy() in a streaming way, * instead of constructing a huge String to concat all the elements: - * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = + * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = * for (e <- record._2){f(e)} * @return the result RDD */ def pipe( - command: Seq[String], - env: Map[String, String] = Map(), + command: Seq[String], + env: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, - printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = - new PipedRDD(this, command, env, - if (printPipeContext ne null) sc.clean(printPipeContext) else null, + printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = + new PipedRDD(this, command, env, + if (printPipeContext ne null) sc.clean(printPipeContext) else null, if (printRDDElement ne null) sc.clean(printRDDElement) else null) /** diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index d8db69b1c9..919bbb2ed8 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -251,4 +251,38 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(topK.size === 2) assert(topK.sorted === Array("b", "a")) } + + test("takeSample") { + sc = new SparkContext("local", "test") + val data = sc.parallelize(1 to 100, 2) + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=false, 20, seed) + assert(sample.size === 20) // Got exactly 20 elements + assert(sample.toSet.size === 20) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=false, 200, seed) + assert(sample.size === 100) // Got only 100 elements + assert(sample.toSet.size === 100) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=true, 20, seed) + assert(sample.size === 20) // Got exactly 20 elements + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=true, 100, seed) + assert(sample.size === 100) // Got exactly 100 elements + // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=true, 200, seed) + assert(sample.size === 200) // Got exactly 200 elements + // Chance of getting all distinct elements is still quite low, so test we got < 100 + assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + } + } } From e7d49388e3df2cd3abb8fedc8a097831d4f33e3e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Jun 2013 00:01:10 -0400 Subject: [PATCH 126/419] Added unit test for K-means, and fixed some bugs --- .../scala/spark/ml/clustering/KMeans.scala | 2 +- .../spark/ml/clustering/KMeansModel.scala | 11 +- .../spark/ml/clustering/KMeansSuite.scala | 150 ++++++++++++++++++ 3 files changed, 152 insertions(+), 11 deletions(-) create mode 100644 ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/ml/src/main/scala/spark/ml/clustering/KMeans.scala index d1786b0812..8c4bec2a26 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeans.scala @@ -226,7 +226,7 @@ class KMeans private ( }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => val myCenters = centers(r).toArray - val myWeights = (0 until myCenters.length).map(i => weightMap((r, i))).toArray + val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) } diff --git a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala index 84bfd0f99a..8244ccc55b 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala @@ -14,16 +14,7 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable /** Return the cluster index that a given point belongs to. */ def predict(point: Array[Double]): Int = { - var bestDist = Double.PositiveInfinity - var bestIndex = -1 - for (i <- 0 until k) { - val dist = MLUtils.squaredDistance(clusterCenters(i), point) - if (dist < bestDist) { - bestDist = dist - bestIndex = i - } - } - bestIndex + KMeans.findClosest(clusterCenters, point)._1 } /** diff --git a/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala b/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala new file mode 100644 index 0000000000..f3bd1d599f --- /dev/null +++ b/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala @@ -0,0 +1,150 @@ +package spark.ml.clustering + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + +import org.jblas._ + + +class KMeansSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + val EPSILON = 1e-4 + + def prettyPrint(point: Array[Double]): String = point.mkString("(", ", ", ")") + + def prettyPrint(points: Array[Array[Double]]): String = { + points.map(prettyPrint).mkString("(", "; ", ")") + } + + // L1 distance between two points + def distance1(v1: Array[Double], v2: Array[Double]): Double = { + v1.zip(v2).map{ case (a, b) => math.abs(a-b) }.max + } + + // Assert that two vectors are equal within tolerance EPSILON + def assertEqual(v1: Array[Double], v2: Array[Double]) { + def errorMessage = prettyPrint(v1) + " did not equal " + prettyPrint(v2) + assert(v1.length == v2.length, errorMessage) + assert(distance1(v1, v2) <= EPSILON, errorMessage) + } + + // Assert that two sets of points are equal, within EPSILON tolerance + def assertSetsEqual(set1: Array[Array[Double]], set2: Array[Array[Double]]) { + def errorMessage = prettyPrint(set1) + " did not equal " + prettyPrint(set2) + assert(set1.length == set2.length, errorMessage) + for (v <- set1) { + val closestDistance = set2.map(w => distance1(v, w)).min + if (closestDistance > EPSILON) { + fail(errorMessage) + } + } + for (v <- set2) { + val closestDistance = set1.map(w => distance1(v, w)).min + if (closestDistance > EPSILON) { + fail(errorMessage) + } + } + } + + test("single cluster") { + val data = sc.parallelize(Array( + Array(1.0, 2.0, 6.0), + Array(1.0, 3.0, 0.0), + Array(1.0, 4.0, 6.0) + )) + + // No matter how many runs or iterations we use, we should get one cluster, + // centered at the mean of the points + + var model = KMeans.train(data, k=1, maxIterations=1) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=2) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode="random") + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode="k-means||") + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + } + + test("single cluster with big dataset") { + val smallData = Array( + Array(1.0, 2.0, 6.0), + Array(1.0, 3.0, 0.0), + Array(1.0, 4.0, 6.0) + ) + val data = sc.parallelize((1 to 100).flatMap(_ => smallData), 4) + + // No matter how many runs or iterations we use, we should get one cluster, + // centered at the mean of the points + + var model = KMeans.train(data, k=1, maxIterations=1) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=2) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode="random") + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode="k-means||") + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + } + + test("k-means|| initialization") { + val points = Array( + Array(1.0, 2.0, 6.0), + Array(1.0, 3.0, 0.0), + Array(1.0, 4.0, 6.0), + Array(1.0, 0.0, 1.0), + Array(1.0, 1.0, 1.0) + ) + val rdd = sc.parallelize(points) + + // K-means|| initialization should place all clusters into distinct centers because + // it will make at least five passes, and it will give non-zero probability to each + // unselected point as long as it hasn't yet selected all of them + + var model = KMeans.train(rdd, k=5, maxIterations=1) + assertSetsEqual(model.clusterCenters, points) + + // Iterations of Lloyd's should not change the answer either + model = KMeans.train(rdd, k=5, maxIterations=10) + assertSetsEqual(model.clusterCenters, points) + + // Neither should more runs + model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) + assertSetsEqual(model.clusterCenters, points) + } +} From 9441d3ef09273436044a8a8b37706eab8df58995 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Jun 2013 00:50:14 -0400 Subject: [PATCH 127/419] Use random seeds for K-means and ALS, and increase tolerance in tests Random seeds make more sense by default for a machine learning library because other libraries behave the same way (people expect to be able to run the algorithm multiple times and get a better answer), but we can add configuration later if needed. Tests that depend on specific seed choices seem brittle. --- ml/src/main/scala/spark/ml/clustering/KMeans.scala | 7 ++++--- ml/src/main/scala/spark/ml/recommendation/ALS.scala | 11 ++++++----- .../test/scala/spark/ml/recommendation/ALSSuite.scala | 4 ++-- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/ml/src/main/scala/spark/ml/clustering/KMeans.scala index 8c4bec2a26..d35f942c01 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeans.scala @@ -179,7 +179,7 @@ class KMeans private ( */ private def initRandom(data: RDD[Array[Double]]): Array[ClusterCenters] = { // Sample all the cluster centers in one pass to avoid repeated scans - val sample = data.takeSample(true, runs * k, 1) + val sample = data.takeSample(true, runs * k, new Random().nextInt()) Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k)) } @@ -194,7 +194,8 @@ class KMeans private ( */ private def initKMeansParallel(data: RDD[Array[Double]]): Array[ClusterCenters] = { // Initialize each run's center to a random point - val sample = data.takeSample(true, runs, 1) + val seed = new Random().nextInt() + val sample = data.takeSample(true, runs, seed) val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r))) // On each step, sample 2 * k points on average for each run with probability proportional @@ -205,7 +206,7 @@ class KMeans private ( for (r <- 0 until runs) yield (r, KMeans.pointCost(centerArrays(r), point)) }.reduceByKey(_ + _).collectAsMap() val chosen = data.mapPartitionsWithIndex { (index, points) => - val rand = new Random((step << 16) ^ index) + val rand = new Random(seed ^ (step << 16) ^ index) for { p <- points r <- 0 until runs diff --git a/ml/src/main/scala/spark/ml/recommendation/ALS.scala b/ml/src/main/scala/spark/ml/recommendation/ALS.scala index abf54216a5..8d5c16847a 100644 --- a/ml/src/main/scala/spark/ml/recommendation/ALS.scala +++ b/ml/src/main/scala/spark/ml/recommendation/ALS.scala @@ -97,9 +97,10 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) - // Initialize user and product factors deterministically. - var users = userOutLinks.mapValues(_.elementIds.map(u => makeInitialFactor(rank, u))) - var products = productOutLinks.mapValues(_.elementIds.map(p => makeInitialFactor(rank, -p))) + // Initialize user and product factors randomly + val seed = new Random().nextInt() + var users = userOutLinks.mapValues(_.elementIds.map(u => randomFactor(rank, seed ^ u))) + var products = productOutLinks.mapValues(_.elementIds.map(p => randomFactor(rank, seed ^ ~p))) for (iter <- 0 until iterations) { // perform ALS update @@ -174,10 +175,10 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l } /** - * Make a random initial factor vector with the given seed. + * Make a random factor vector with the given seed. * TODO: Initialize things using mapPartitionsWithIndex to make it faster? */ - private def makeInitialFactor(rank: Int, seed: Int): Array[Double] = { + private def randomFactor(rank: Int, seed: Int): Array[Double] = { val rand = new Random(seed) Array.fill(rank)(rand.nextDouble) } diff --git a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala index c450d5315c..f3f56c4357 100644 --- a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala +++ b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -20,11 +20,11 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll { } test("rank-1 matrices") { - testALS(10, 20, 1, 5, 0.5, 0.1) + testALS(10, 20, 1, 15, 0.7, 0.3) } test("rank-2 matrices") { - testALS(20, 30, 2, 10, 0.7, 0.3) + testALS(20, 30, 2, 15, 0.7, 0.3) } /** From 09f187a400623020b520a4cc5c8fc5814746dab1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 30 Jun 2013 14:40:24 -0700 Subject: [PATCH 128/419] Add top-level methods for regression methods. Also add multiple versions of them to make it easier to call them from java. --- .../ml/regression/LogisticRegression.scala | 64 ++++++++++++++++++- .../spark/ml/regression/RidgeRegression.scala | 40 ++++++++++-- 2 files changed, 96 insertions(+), 8 deletions(-) diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala index 1508f6934a..435669f72a 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -83,8 +83,67 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } } +/** + * Top-level methods for calling Logistic Regression. + */ object LogisticRegression { + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double) + : LogisticRegressionModel = + { + new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(input) + } + + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LogisticRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double) + : LogisticRegressionModel = + { + train(input, numIterations, stepSize, 1.0) + } + + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LogisticRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int) + : LogisticRegressionModel = + { + train(input, numIterations, 1.0, 1.0) + } + def main(args: Array[String]) { if (args.length != 4) { println("Usage: LogisticRegression ") @@ -92,9 +151,8 @@ object LogisticRegression { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadData(sc, args(1)) - val lr = new LogisticRegression().setStepSize(args(2).toDouble) - .setNumIterations(args(3).toInt) - val model = lr.train(data) + val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) + sc.stop() } } diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala index 16d146ac27..dae224144e 100644 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -134,9 +134,42 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) model } } - +/** + * Top-level methods for calling Ridge Regression. + */ object RidgeRegression { + /** + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for + * a given lambda. The optimal lambda is computed by performing binary search + * between the provided bounds of lambda. + * + * @param input RDD of (response, array of features) pairs. + * @param lambdaLow lower bound used in binary search for lambda + * @param lambdaHigh upper bound used in binary search for lambda + */ + def train( + input: RDD[(Double, Array[Double])], + lambdaLow: Double, + lambdaHigh: Double) + : RidgeRegressionModel = + { + new RidgeRegression(lambdaLow, lambdaHigh).train(input) + } + + /** + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for + * a given lambda. The optimal lambda is computed by performing binary search + * between lambda values of 0 and 100. + * + * @param input RDD of (response, array of features) pairs. + */ + def train(input: RDD[(Double, Array[Double])]) : RidgeRegressionModel = { + train(input, 0.0, 100.0) + } + def main(args: Array[String]) { if (args.length != 2) { println("Usage: RidgeRegression ") @@ -144,10 +177,7 @@ object RidgeRegression { } val sc = new SparkContext(args(0), "RidgeRegression") val data = MLUtils.loadData(sc, args(1)) - val ridgeReg = new RidgeRegression().setLowLambda(0) - .setHighLambda(1000) - - val model = ridgeReg.train(data) + val model = RidgeRegression.train(data, 0, 1000) sc.stop() } } From 0e33c88cbd85d3c9aa73b4c66a03372169abf3a1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 30 Jun 2013 14:43:08 -0700 Subject: [PATCH 129/419] Rename package gradient to optimization --- .../scala/spark/ml/{gradient => optimization}/Gradient.scala | 2 +- .../spark/ml/{gradient => optimization}/GradientDescent.scala | 2 +- .../scala/spark/ml/{gradient => optimization}/Updater.scala | 2 +- ml/src/main/scala/spark/ml/regression/LogisticRegression.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) rename ml/src/main/scala/spark/ml/{gradient => optimization}/Gradient.scala (97%) rename ml/src/main/scala/spark/ml/{gradient => optimization}/GradientDescent.scala (98%) rename ml/src/main/scala/spark/ml/{gradient => optimization}/Updater.scala (97%) diff --git a/ml/src/main/scala/spark/ml/gradient/Gradient.scala b/ml/src/main/scala/spark/ml/optimization/Gradient.scala similarity index 97% rename from ml/src/main/scala/spark/ml/gradient/Gradient.scala rename to ml/src/main/scala/spark/ml/optimization/Gradient.scala index 50795c01a2..6d062ebddf 100644 --- a/ml/src/main/scala/spark/ml/gradient/Gradient.scala +++ b/ml/src/main/scala/spark/ml/optimization/Gradient.scala @@ -1,4 +1,4 @@ -package spark.ml.gradient +package spark.ml.optimization import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala b/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala similarity index 98% rename from ml/src/main/scala/spark/ml/gradient/GradientDescent.scala rename to ml/src/main/scala/spark/ml/optimization/GradientDescent.scala index 8cd2a69a3a..d959ebf71c 100644 --- a/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala +++ b/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala @@ -1,4 +1,4 @@ -package spark.ml.gradient +package spark.ml.optimization import spark.{Logging, RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/main/scala/spark/ml/gradient/Updater.scala b/ml/src/main/scala/spark/ml/optimization/Updater.scala similarity index 97% rename from ml/src/main/scala/spark/ml/gradient/Updater.scala rename to ml/src/main/scala/spark/ml/optimization/Updater.scala index 27f03897b0..dfc7bf2025 100644 --- a/ml/src/main/scala/spark/ml/gradient/Updater.scala +++ b/ml/src/main/scala/spark/ml/optimization/Updater.scala @@ -1,4 +1,4 @@ -package spark.ml.gradient +package spark.ml.optimization import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala index 435669f72a..3c471ab652 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -1,7 +1,7 @@ package spark.ml.regression import spark.{Logging, RDD, SparkContext} -import spark.ml.gradient._ +import spark.ml.optimization._ import spark.ml.util.MLUtils import org.jblas.DoubleMatrix From 399bd65ef5f780c2796f6facf9fac8fd9ec2c2f4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:27:06 -0700 Subject: [PATCH 130/419] Fixed compile error due to merge --- core/src/test/scala/spark/RDDSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 919bbb2ed8..e41ae385c0 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -253,7 +253,6 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("takeSample") { - sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 100, 2) for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) From 43b24635ee45a845f2432bc13c11fcf2eb02f2f3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:38:53 -0700 Subject: [PATCH 131/419] Renamed ML package to MLlib and added it to classpath --- bin/compute-classpath.cmd | 2 ++ bin/compute-classpath.sh | 2 ++ {ml => mllib}/data/als/test.data | 0 {ml => mllib}/data/lr-data/random.data | 0 {ml => mllib}/data/ridge-data/lpsa.data | 0 .../src/main/scala/spark/ml/clustering/KMeans.scala | 9 +++++---- .../src/main/scala/spark/ml/clustering/KMeansModel.scala | 4 ++-- .../src/main/scala/spark/ml/clustering/LocalKMeans.scala | 4 ++-- .../src/main/scala/spark/ml/optimization/Gradient.scala | 2 +- .../scala/spark/ml/optimization/GradientDescent.scala | 2 +- .../src/main/scala/spark/ml/optimization/Updater.scala | 2 +- .../src/main/scala/spark/ml/recommendation/ALS.scala | 2 +- .../ml/recommendation/MatrixFactorizationModel.scala | 2 +- .../scala/spark/ml/regression/LogisticRegression.scala | 6 +++--- .../ml/regression/LogisticRegressionGenerator.scala | 4 ++-- .../src/main/scala/spark/ml/regression/Regression.scala | 2 +- .../main/scala/spark/ml/regression/RidgeRegression.scala | 4 ++-- .../spark/ml/regression/RidgeRegressionGenerator.scala | 4 ++-- {ml => mllib}/src/main/scala/spark/ml/util/MLUtils.scala | 2 +- {ml => mllib}/src/test/resources/log4j.properties | 0 .../src/test/scala/spark/ml/clustering/KMeansSuite.scala | 2 +- .../test/scala/spark/ml/recommendation/ALSSuite.scala | 2 +- .../spark/ml/regression/LogisticRegressionSuite.scala | 2 +- .../scala/spark/ml/regression/RidgeRegressionSuite.scala | 2 +- project/SparkBuild.scala | 8 ++++---- 25 files changed, 37 insertions(+), 32 deletions(-) rename {ml => mllib}/data/als/test.data (100%) rename {ml => mllib}/data/lr-data/random.data (100%) rename {ml => mllib}/data/ridge-data/lpsa.data (100%) rename {ml => mllib}/src/main/scala/spark/ml/clustering/KMeans.scala (97%) rename {ml => mllib}/src/main/scala/spark/ml/clustering/KMeansModel.scala (92%) rename {ml => mllib}/src/main/scala/spark/ml/clustering/LocalKMeans.scala (97%) rename {ml => mllib}/src/main/scala/spark/ml/optimization/Gradient.scala (96%) rename {ml => mllib}/src/main/scala/spark/ml/optimization/GradientDescent.scala (98%) rename {ml => mllib}/src/main/scala/spark/ml/optimization/Updater.scala (96%) rename {ml => mllib}/src/main/scala/spark/ml/recommendation/ALS.scala (99%) rename {ml => mllib}/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala (95%) rename {ml => mllib}/src/main/scala/spark/ml/regression/LogisticRegression.scala (98%) rename {ml => mllib}/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala (94%) rename {ml => mllib}/src/main/scala/spark/ml/regression/Regression.scala (94%) rename {ml => mllib}/src/main/scala/spark/ml/regression/RidgeRegression.scala (98%) rename {ml => mllib}/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala (96%) rename {ml => mllib}/src/main/scala/spark/ml/util/MLUtils.scala (99%) rename {ml => mllib}/src/test/resources/log4j.properties (100%) rename {ml => mllib}/src/test/scala/spark/ml/clustering/KMeansSuite.scala (99%) rename {ml => mllib}/src/test/scala/spark/ml/recommendation/ALSSuite.scala (98%) rename {ml => mllib}/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala (98%) rename {ml => mllib}/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala (97%) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 6e7efbd334..44826f339c 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -15,6 +15,7 @@ set CORE_DIR=%FWDIR%core set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel +set MLLIB_DIR=%FWDIR%mllib set STREAMING_DIR=%FWDIR%streaming set PYSPARK_DIR=%FWDIR%python @@ -29,6 +30,7 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\* set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%MLLIB_DIR%\target\scala-%SCALA_VERSION%\classes rem Add hadoop conf dir - else FileSystem.*, etc fail rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 3a78880290..75c58d1181 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -18,6 +18,7 @@ REPL_DIR="$FWDIR/repl" REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" +MLLIB_DIR="$FWDIR/mllib" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -49,6 +50,7 @@ if [ -e $REPL_BIN_DIR/target ]; then CLASSPATH+=":$EXAMPLES_JAR" fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH="$CLASSPATH:$jar" done diff --git a/ml/data/als/test.data b/mllib/data/als/test.data similarity index 100% rename from ml/data/als/test.data rename to mllib/data/als/test.data diff --git a/ml/data/lr-data/random.data b/mllib/data/lr-data/random.data similarity index 100% rename from ml/data/lr-data/random.data rename to mllib/data/lr-data/random.data diff --git a/ml/data/ridge-data/lpsa.data b/mllib/data/ridge-data/lpsa.data similarity index 100% rename from ml/data/ridge-data/lpsa.data rename to mllib/data/ridge-data/lpsa.data diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala similarity index 97% rename from ml/src/main/scala/spark/ml/clustering/KMeans.scala rename to mllib/src/main/scala/spark/ml/clustering/KMeans.scala index d35f942c01..6d78f926c2 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala @@ -1,4 +1,4 @@ -package spark.ml.clustering +package spark.mllib.clustering import scala.collection.mutable.ArrayBuffer import scala.util.Random @@ -6,7 +6,7 @@ import scala.util.Random import spark.{SparkContext, RDD} import spark.SparkContext._ import spark.Logging -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils import org.jblas.DoubleMatrix @@ -270,7 +270,8 @@ object KMeans { /** * Return the index of the closest point in `centers` to `point`, as well as its distance. */ - private[ml] def findClosest(centers: Array[Array[Double]], point: Array[Double]): (Int, Double) = + private[mllib] def findClosest(centers: Array[Array[Double]], point: Array[Double]) + : (Int, Double) = { var bestDistance = Double.PositiveInfinity var bestIndex = 0 @@ -287,7 +288,7 @@ object KMeans { /** * Return the K-means cost of a given point against the given cluster centers. */ - private[ml] def pointCost(centers: Array[Array[Double]], point: Array[Double]): Double = { + private[mllib] def pointCost(centers: Array[Array[Double]], point: Array[Double]): Double = { var bestDistance = Double.PositiveInfinity for (i <- 0 until centers.length) { val distance = MLUtils.squaredDistance(point, centers(i)) diff --git a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala similarity index 92% rename from ml/src/main/scala/spark/ml/clustering/KMeansModel.scala rename to mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala index 8244ccc55b..4fd0646160 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala @@ -1,8 +1,8 @@ -package spark.ml.clustering +package spark.mllib.clustering import spark.RDD import spark.SparkContext._ -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils /** diff --git a/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala similarity index 97% rename from ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala rename to mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala index 03129ef552..e12b3be251 100644 --- a/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala @@ -1,4 +1,4 @@ -package spark.ml.clustering +package spark.mllib.clustering import scala.util.Random @@ -8,7 +8,7 @@ import org.jblas.{DoubleMatrix, SimpleBlas} * An utility object to run K-means locally. This is private to the ML package because it's used * in the initialization of KMeans but not meant to be publicly exposed. */ -private[ml] object LocalKMeans { +private[mllib] object LocalKMeans { /** * Run K-means++ on the weighted point set `points`. This first does the K-means++ * initialization procedure and then roudns of Lloyd's algorithm. diff --git a/ml/src/main/scala/spark/ml/optimization/Gradient.scala b/mllib/src/main/scala/spark/ml/optimization/Gradient.scala similarity index 96% rename from ml/src/main/scala/spark/ml/optimization/Gradient.scala rename to mllib/src/main/scala/spark/ml/optimization/Gradient.scala index 6d062ebddf..90b0999a5e 100644 --- a/ml/src/main/scala/spark/ml/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/ml/optimization/Gradient.scala @@ -1,4 +1,4 @@ -package spark.ml.optimization +package spark.mllib.optimization import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala similarity index 98% rename from ml/src/main/scala/spark/ml/optimization/GradientDescent.scala rename to mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala index d959ebf71c..eff853f379 100644 --- a/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala @@ -1,4 +1,4 @@ -package spark.ml.optimization +package spark.mllib.optimization import spark.{Logging, RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/main/scala/spark/ml/optimization/Updater.scala b/mllib/src/main/scala/spark/ml/optimization/Updater.scala similarity index 96% rename from ml/src/main/scala/spark/ml/optimization/Updater.scala rename to mllib/src/main/scala/spark/ml/optimization/Updater.scala index dfc7bf2025..ea80bfcbfd 100644 --- a/ml/src/main/scala/spark/ml/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/ml/optimization/Updater.scala @@ -1,4 +1,4 @@ -package spark.ml.optimization +package spark.mllib.optimization import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala similarity index 99% rename from ml/src/main/scala/spark/ml/recommendation/ALS.scala rename to mllib/src/main/scala/spark/ml/recommendation/ALS.scala index 8d5c16847a..0c6fa6f741 100644 --- a/ml/src/main/scala/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala @@ -1,4 +1,4 @@ -package spark.ml.recommendation +package spark.mllib.recommendation import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random diff --git a/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala similarity index 95% rename from ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala rename to mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala index cfdf2ba523..fb812a6dbe 100644 --- a/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala @@ -1,4 +1,4 @@ -package spark.ml.recommendation +package spark.mllib.recommendation import spark.RDD import spark.SparkContext._ diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala similarity index 98% rename from ml/src/main/scala/spark/ml/regression/LogisticRegression.scala rename to mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala index 3c471ab652..448ab9dce9 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -1,8 +1,8 @@ -package spark.ml.regression +package spark.mllib.regression import spark.{Logging, RDD, SparkContext} -import spark.ml.optimization._ -import spark.ml.util.MLUtils +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala similarity index 94% rename from ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala index 6d37aad047..9f6abab70b 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala @@ -1,11 +1,11 @@ -package spark.ml.regression +package spark.mllib.regression import scala.util.Random import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils object LogisticRegressionGenerator { diff --git a/ml/src/main/scala/spark/ml/regression/Regression.scala b/mllib/src/main/scala/spark/ml/regression/Regression.scala similarity index 94% rename from ml/src/main/scala/spark/ml/regression/Regression.scala rename to mllib/src/main/scala/spark/ml/regression/Regression.scala index 4a20f513b7..f79974c191 100644 --- a/ml/src/main/scala/spark/ml/regression/Regression.scala +++ b/mllib/src/main/scala/spark/ml/regression/Regression.scala @@ -1,4 +1,4 @@ -package spark.ml.regression +package spark.mllib.regression import spark.RDD diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala similarity index 98% rename from ml/src/main/scala/spark/ml/regression/RidgeRegression.scala rename to mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala index dae224144e..2d07c77141 100644 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -1,8 +1,8 @@ -package spark.ml.regression +package spark.mllib.regression import spark.{Logging, RDD, SparkContext} import spark.SparkContext._ -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils import org.jblas.DoubleMatrix import org.jblas.Solve diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala similarity index 96% rename from ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala index 75854fe1de..c9ac4a8b07 100644 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala @@ -1,11 +1,11 @@ -package spark.ml.regression +package spark.mllib.regression import scala.util.Random import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils object RidgeRegressionGenerator { diff --git a/ml/src/main/scala/spark/ml/util/MLUtils.scala b/mllib/src/main/scala/spark/ml/util/MLUtils.scala similarity index 99% rename from ml/src/main/scala/spark/ml/util/MLUtils.scala rename to mllib/src/main/scala/spark/ml/util/MLUtils.scala index 6efa7c81ad..0a4a037c71 100644 --- a/ml/src/main/scala/spark/ml/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/ml/util/MLUtils.scala @@ -1,4 +1,4 @@ -package spark.ml.util +package spark.mllib.util import spark.{RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties similarity index 100% rename from ml/src/test/resources/log4j.properties rename to mllib/src/test/resources/log4j.properties diff --git a/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala similarity index 99% rename from ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala rename to mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala index f3bd1d599f..ae7cf57c42 100644 --- a/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.clustering +package spark.mllib.clustering import scala.util.Random diff --git a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala rename to mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala index f3f56c4357..2ada9ae76b 100644 --- a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.recommendation +package spark.mllib.recommendation import scala.util.Random diff --git a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala rename to mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index 55f2c5c18e..04d3400cb4 100644 --- a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.regression +package spark.mllib.regression import scala.util.Random diff --git a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala similarity index 97% rename from ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala rename to mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala index aed5cbec24..df41dbbdff 100644 --- a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.regression +package spark.mllib.regression import scala.util.Random diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5dbb5d4a65..c487f34d4a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -25,7 +25,7 @@ object SparkBuild extends Build { //val HADOOP_MAJOR_VERSION = "2" //val HADOOP_YARN = true - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, ml) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -37,7 +37,7 @@ object SparkBuild extends Build { lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) - lazy val ml = Project("ml", file("ml"), settings = mlSettings) dependsOn (core) + lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn (core) // A configuration to set an alternative publishLocalConfiguration lazy val MavenCompile = config("m2r") extend(Compile) @@ -221,8 +221,8 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") - def mlSettings = sharedSettings ++ Seq( - name := "spark-ml", + def mllibSettings = sharedSettings ++ Seq( + name := "spark-mllib", libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3" ) From de67deeaabc12a62dadee5ec302fe58bee0b3498 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 17:16:49 -0700 Subject: [PATCH 132/419] Addressed style comments from Ryan LeCompte --- .../main/scala/spark/ml/clustering/KMeans.scala | 14 ++++++-------- .../main/scala/spark/ml/recommendation/ALS.scala | 4 +++- .../spark/ml/regression/RidgeRegression.scala | 8 ++++---- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala index 6d78f926c2..b0e141ff32 100644 --- a/mllib/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala @@ -100,7 +100,7 @@ class KMeans private ( val sc = data.sparkContext - var centers = if (initializationMode == KMeans.RANDOM) { + val centers = if (initializationMode == KMeans.RANDOM) { initRandom(data) } else { initKMeansParallel(data) @@ -131,13 +131,11 @@ class KMeans private ( val sums = Array.fill(runs, k)(new DoubleMatrix(dims)) val counts = Array.fill(runs, k)(0L) - for (point <- points) { - for ((centers, runIndex) <- activeCenters.zipWithIndex) { - val (bestCenter, cost) = KMeans.findClosest(centers, point) - costAccums(runIndex) += cost - sums(runIndex)(bestCenter).addi(new DoubleMatrix(point)) - counts(runIndex)(bestCenter) += 1 - } + for (point <- points; (centers, runIndex) <- activeCenters.zipWithIndex) { + val (bestCenter, cost) = KMeans.findClosest(centers, point) + costAccums(runIndex) += cost + sums(runIndex)(bestCenter).addi(new DoubleMatrix(point)) + counts(runIndex)(bestCenter) += 1 } val contribs = for (i <- 0 until runs; j <- 0 until k) yield { diff --git a/mllib/src/main/scala/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala index 0c6fa6f741..6c9fb2359c 100644 --- a/mllib/src/main/scala/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala @@ -168,7 +168,9 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { val ratings = elements.map(_._2).toArray - Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) + val inLinkBlock = makeInLinkBlock(numBlocks, ratings) + val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) + Iterator.single((blockId, (inLinkBlock, outLinkBlock))) }, true) links.persist(StorageLevel.MEMORY_AND_DISK) (links.mapValues(_._1), links.mapValues(_._2)) diff --git a/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala index 2d07c77141..a6ececbeb6 100644 --- a/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -140,8 +140,8 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) object RidgeRegression { /** - * Train a ridge regression model given an RDD of (response, features) pairs. - * We use the closed form solution to compute the cross-validation score for + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for * a given lambda. The optimal lambda is computed by performing binary search * between the provided bounds of lambda. * @@ -159,8 +159,8 @@ object RidgeRegression { } /** - * Train a ridge regression model given an RDD of (response, features) pairs. - * We use the closed form solution to compute the cross-validation score for + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for * a given lambda. The optimal lambda is computed by performing binary search * between lambda values of 0 and 100. * From 653043beb6681c57a02a3e8dde837d7dbc1e44bf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 17:18:55 -0700 Subject: [PATCH 133/419] Renamed files to match package --- mllib/src/main/scala/spark/{ml => mllib}/clustering/KMeans.scala | 0 .../main/scala/spark/{ml => mllib}/clustering/KMeansModel.scala | 0 .../main/scala/spark/{ml => mllib}/clustering/LocalKMeans.scala | 0 .../main/scala/spark/{ml => mllib}/optimization/Gradient.scala | 0 .../scala/spark/{ml => mllib}/optimization/GradientDescent.scala | 0 .../src/main/scala/spark/{ml => mllib}/optimization/Updater.scala | 0 mllib/src/main/scala/spark/{ml => mllib}/recommendation/ALS.scala | 0 .../{ml => mllib}/recommendation/MatrixFactorizationModel.scala | 0 .../scala/spark/{ml => mllib}/regression/LogisticRegression.scala | 0 .../{ml => mllib}/regression/LogisticRegressionGenerator.scala | 0 .../main/scala/spark/{ml => mllib}/regression/Regression.scala | 0 .../scala/spark/{ml => mllib}/regression/RidgeRegression.scala | 0 .../spark/{ml => mllib}/regression/RidgeRegressionGenerator.scala | 0 mllib/src/main/scala/spark/{ml => mllib}/util/MLUtils.scala | 0 .../test/scala/spark/{ml => mllib}/clustering/KMeansSuite.scala | 0 .../test/scala/spark/{ml => mllib}/recommendation/ALSSuite.scala | 0 .../spark/{ml => mllib}/regression/LogisticRegressionSuite.scala | 0 .../spark/{ml => mllib}/regression/RidgeRegressionSuite.scala | 0 18 files changed, 0 insertions(+), 0 deletions(-) rename mllib/src/main/scala/spark/{ml => mllib}/clustering/KMeans.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/clustering/KMeansModel.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/clustering/LocalKMeans.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/optimization/Gradient.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/optimization/GradientDescent.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/optimization/Updater.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/recommendation/ALS.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/recommendation/MatrixFactorizationModel.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/LogisticRegression.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/LogisticRegressionGenerator.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/Regression.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/RidgeRegression.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/RidgeRegressionGenerator.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/util/MLUtils.scala (100%) rename mllib/src/test/scala/spark/{ml => mllib}/clustering/KMeansSuite.scala (100%) rename mllib/src/test/scala/spark/{ml => mllib}/recommendation/ALSSuite.scala (100%) rename mllib/src/test/scala/spark/{ml => mllib}/regression/LogisticRegressionSuite.scala (100%) rename mllib/src/test/scala/spark/{ml => mllib}/regression/RidgeRegressionSuite.scala (100%) diff --git a/mllib/src/main/scala/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/clustering/KMeans.scala rename to mllib/src/main/scala/spark/mllib/clustering/KMeans.scala diff --git a/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala rename to mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala diff --git a/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala rename to mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala diff --git a/mllib/src/main/scala/spark/ml/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/optimization/Gradient.scala rename to mllib/src/main/scala/spark/mllib/optimization/Gradient.scala diff --git a/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala rename to mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala diff --git a/mllib/src/main/scala/spark/ml/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/optimization/Updater.scala rename to mllib/src/main/scala/spark/mllib/optimization/Updater.scala diff --git a/mllib/src/main/scala/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/recommendation/ALS.scala rename to mllib/src/main/scala/spark/mllib/recommendation/ALS.scala diff --git a/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala rename to mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala diff --git a/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala rename to mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala diff --git a/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala diff --git a/mllib/src/main/scala/spark/ml/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/Regression.scala rename to mllib/src/main/scala/spark/mllib/regression/Regression.scala diff --git a/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala rename to mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala diff --git a/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala diff --git a/mllib/src/main/scala/spark/ml/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/util/MLUtils.scala rename to mllib/src/main/scala/spark/mllib/util/MLUtils.scala diff --git a/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala similarity index 100% rename from mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala rename to mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala diff --git a/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala similarity index 100% rename from mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala rename to mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala diff --git a/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala similarity index 100% rename from mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala rename to mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala diff --git a/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala similarity index 100% rename from mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala rename to mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala From 84b7fc54e6777167f96742cd61326581f342fb03 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 5 Jul 2013 17:21:01 -0700 Subject: [PATCH 134/419] Enforcing correct sort order for formatted strings --- .../main/scala/spark/deploy/master/ui/IndexPage.scala | 10 +++++++--- .../main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 +++- core/src/main/scala/spark/ui/jobs/StagePage.scala | 4 +++- core/src/main/scala/spark/ui/storage/RDDPage.scala | 8 ++++++-- 4 files changed, 19 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 2ed566a5bc..c6de2bafa3 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -90,8 +90,10 @@ private[spark] class IndexPage(parent: MasterWebUI) { {worker.host}:{worker.port} {worker.state} {worker.cores} ({worker.coresUsed} Used) - {Utils.memoryMegabytesToString(worker.memory)} - ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) + + {Utils.memoryMegabytesToString(worker.memory)} + ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) + } @@ -105,7 +107,9 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.coresGranted} - {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} + + {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} + {DeployWebUI.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index c65d5b4faf..e466129c1a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -77,7 +77,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { {executor.execId} {executor.cores} - {Utils.memoryMegabytesToString(executor.memory)} + + {Utils.memoryMegabytesToString(executor.memory)} +
    • ID: {executor.appId}
    • diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index c6f87fc652..49e84880cf 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -93,7 +93,9 @@ private[spark] class StagePage(parent: JobProgressUI) { val (info, metrics, exception) = taskData {info.taskId} - {Option(metrics).map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} + m.executorRunTime.toString}.getOrElse("1")}> + {Option(metrics).map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} + {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 65952f711a..0cb1e47ea5 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -81,8 +81,12 @@ private[spark] class RDDPage(parent: BlockManagerUI) { {block.storageLevel.description} - {Utils.memoryBytesToString(block.memSize)} - {Utils.memoryBytesToString(block.diskSize)} + + {Utils.memoryBytesToString(block.memSize)} + + + {Utils.memoryBytesToString(block.diskSize)} + } From 8bbe907556041443a411b69c95d7a9cd3eb69dcc Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 17:25:23 -0700 Subject: [PATCH 135/419] Replaced string constants in test --- .../scala/spark/mllib/clustering/KMeansSuite.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala index ae7cf57c42..cb096f39a9 100644 --- a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala @@ -21,6 +21,8 @@ class KMeansSuite extends FunSuite with BeforeAndAfterAll { val EPSILON = 1e-4 + import KMeans.{RANDOM, K_MEANS_PARALLEL} + def prettyPrint(point: Array[Double]): String = point.mkString("(", ", ", ")") def prettyPrint(points: Array[Array[Double]]): String = { @@ -82,10 +84,11 @@ class KMeansSuite extends FunSuite with BeforeAndAfterAll { model = KMeans.train(data, k=1, maxIterations=1, runs=5) assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode="random") + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode="k-means||") + model = KMeans.train( + data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) } @@ -115,10 +118,10 @@ class KMeansSuite extends FunSuite with BeforeAndAfterAll { model = KMeans.train(data, k=1, maxIterations=1, runs=5) assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode="random") + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode="k-means||") + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) } From 757e56dfc7bd900d5b3f3f145eabe8198bfbe7cc Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Fri, 5 Jul 2013 19:54:28 -0700 Subject: [PATCH 136/419] make binSearch a tail-recursive method --- .../mllib/regression/RidgeRegression.scala | 37 ++++++++++++------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index a6ececbeb6..8343f28139 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -1,12 +1,13 @@ package spark.mllib.regression import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ import spark.mllib.util.MLUtils import org.jblas.DoubleMatrix import org.jblas.Solve +import scala.annotation.tailrec + /** * Ridge Regression from Joseph Gonzalez's implementation in MLBase */ @@ -99,20 +100,28 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) // Binary search for the best assignment to lambda. def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { - val mid = (high - low) / 2 + low - val lowValue = crossValidate((mid - low) / 2 + low) - val highValue = crossValidate((high - mid) / 2 + mid) - val (newLow, newHigh) = if (lowValue._2 < highValue._2) { - (low, mid + (high-low)/4) - } else { - (mid - (high-low)/4, high) - } - if (newHigh - newLow > 1.0E-7) { - // :: is list prepend in Scala. - lowValue :: highValue :: binSearch(newLow, newHigh) - } else { - List(lowValue, highValue) + @tailrec + def loop( + low: Double, + high: Double, + acc: List[(Double, Double, DoubleMatrix)]): List[(Double, Double, DoubleMatrix)] = { + val mid = (high - low) / 2 + low + val lowValue = crossValidate((mid - low) / 2 + low) + val highValue = crossValidate((high - mid) / 2 + mid) + val (newLow, newHigh) = if (lowValue._2 < highValue._2) { + (low, mid + (high-low)/4) + } else { + (mid - (high-low)/4, high) + } + if (newHigh - newLow > 1.0E-7) { + // :: is list prepend in Scala. + loop(newLow, newHigh, lowValue :: highValue :: acc) + } else { + lowValue :: highValue :: acc + } } + + loop(low, high, Nil) } // Actually compute the best lambda From 280418ac452b029b15a83d8e2fe05a96417294d1 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 5 Jul 2013 21:38:21 -0700 Subject: [PATCH 137/419] Reduced the number of Iterator to ArrayBuffer copies in NetworkReceiver. --- .../dstream/NetworkInputDStream.scala | 18 ++++++------------ .../streaming/receivers/ActorReceiver.scala | 7 +++++-- 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 26805e9621..122a529bb7 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -140,12 +140,10 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log /** - * Pushes a block (as iterator of values) into the block manager. + * Pushes a block (as an ArrayBuffer filled with data) into the block manager. */ - def pushBlock(blockId: String, iterator: Iterator[T], metadata: Any, level: StorageLevel) { - val buffer = new ArrayBuffer[T] ++ iterator - env.blockManager.put(blockId, buffer.asInstanceOf[ArrayBuffer[Any]], level) - + def pushBlock(blockId: String, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { + env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) actor ! ReportBlock(blockId, metadata) } @@ -195,7 +193,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log class BlockGenerator(storageLevel: StorageLevel) extends Serializable with Logging { - case class Block(id: String, iterator: Iterator[T], metadata: Any = null) + case class Block(id: String, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong @@ -222,17 +220,13 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log currentBuffer += obj } - private def createBlock(blockId: String, iterator: Iterator[T]) : Block = { - new Block(blockId, iterator) - } - private def updateCurrentBuffer(time: Long) { try { val newBlockBuffer = currentBuffer currentBuffer = new ArrayBuffer[T] if (newBlockBuffer.size > 0) { val blockId = "input-" + NetworkReceiver.this.streamId + "-" + (time - blockInterval) - val newBlock = createBlock(blockId, newBlockBuffer.toIterator) + val newBlock = new Block(blockId, newBlockBuffer) blocksForPushing.add(newBlock) } } catch { @@ -248,7 +242,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log try { while(true) { val block = blocksForPushing.take() - NetworkReceiver.this.pushBlock(block.id, block.iterator, block.metadata, storageLevel) + NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) } } catch { case ie: InterruptedException => diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index b3201d0b28..036c95a860 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -9,6 +9,8 @@ import spark.streaming.dstream.NetworkReceiver import java.util.concurrent.atomic.AtomicInteger +import scala.collection.mutable.ArrayBuffer + /** A helper with set of defaults for supervisor strategy **/ object ReceiverSupervisorStrategy { @@ -136,8 +138,9 @@ private[streaming] class ActorReceiver[T: ClassManifest]( } protected def pushBlock(iter: Iterator[T]) { - pushBlock("block-" + streamId + "-" + System.nanoTime(), - iter, null, storageLevel) + val buffer = new ArrayBuffer[T] + buffer ++= iter + pushBlock("block-" + streamId + "-" + System.nanoTime(), buffer, null, storageLevel) } protected def onStart() = { From 37abe84212c6dad6fb87a3b47666d6a3c14c1f66 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 5 Jul 2013 22:54:06 -0700 Subject: [PATCH 138/419] Tracking some task metrics even during failures. --- core/src/main/scala/spark/TaskEndReason.scala | 4 +++- core/src/main/scala/spark/executor/Executor.scala | 10 ++++++++-- .../main/scala/spark/scheduler/DAGScheduler.scala | 2 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../spark/scheduler/local/LocalScheduler.scala | 14 ++++++++++---- .../scheduler/local/LocalTaskSetManager.scala | 2 +- .../main/scala/spark/ui/UIWorkloadGenerator.scala | 12 +++++++++++- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 8 ++++---- 8 files changed, 39 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index 8140cba084..bb75ec208c 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -1,5 +1,6 @@ package spark +import spark.executor.TaskMetrics import spark.storage.BlockManagerId /** @@ -24,7 +25,8 @@ private[spark] case class FetchFailed( private[spark] case class ExceptionFailure( className: String, description: String, - stackTrace: Array[StackTraceElement]) + stackTrace: Array[StackTraceElement], + metrics: Option[TaskMetrics]) extends TaskEndReason private[spark] case class OtherFailure(message: String) extends TaskEndReason diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 8bebfafce4..f01be68d14 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -92,15 +92,18 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val ser = SparkEnv.get.closureSerializer.newInstance() logInfo("Running task ID " + taskId) context.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) + var attemptedTask: Option[Task[Any]] = None + var taskStart: Long = 0 try { SparkEnv.set(env) Accumulators.clear() val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) updateDependencies(taskFiles, taskJars) val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) + attemptedTask = Some(task) logInfo("Its generation is " + task.generation) env.mapOutputTracker.updateGeneration(task.generation) - val taskStart = System.currentTimeMillis() + taskStart = System.currentTimeMillis() val value = task.run(taskId.toInt) val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => @@ -128,7 +131,10 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } case t: Throwable => { - val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) + val serviceTime = (System.currentTimeMillis() - taskStart).toInt + val metrics = attemptedTask.flatMap(t => t.metrics) + metrics.foreach{m => m.executorRunTime = serviceTime} + val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) // TODO: Should we exit the whole executor here? On the one hand, the failed task may diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index d9ddc41aa2..1945a4a514 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -618,7 +618,7 @@ class DAGScheduler( handleExecutorLost(bmAddress.executorId, Some(task.generation)) } - case ExceptionFailure(className, description, stackTrace) => + case ExceptionFailure(className, description, stackTrace, metrics) => // Do nothing here, left up to the TaskScheduler to decide how to handle user failures case other => diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 6965cde5da..fe6420a522 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -571,7 +571,7 @@ private[spark] class ClusterTaskSetManager( return case ef: ExceptionFailure => - sched.listener.taskEnded(tasks(index), ef, null, null, info, null) + sched.listener.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null)) val key = ef.description val now = System.currentTimeMillis val (printFull, dupCount) = { diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 9d375e1db8..b000e328e6 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -145,6 +145,9 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: // Set the Spark execution environment for the worker thread SparkEnv.set(env) val ser = SparkEnv.get.closureSerializer.newInstance() + var attemptedTask: Option[Task[_]] = None + val start = System.currentTimeMillis() + var taskStart: Long = 0 try { Accumulators.clear() Thread.currentThread().setContextClassLoader(classLoader) @@ -153,10 +156,11 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: // this adds a bit of unnecessary overhead but matches how the Mesos Executor works. val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes) updateDependencies(taskFiles, taskJars) // Download any files added with addFile - val taskStart = System.currentTimeMillis() val deserializedTask = ser.deserialize[Task[_]]( taskBytes, Thread.currentThread.getContextClassLoader) - val deserTime = System.currentTimeMillis() - taskStart + attemptedTask = Some(deserializedTask) + val deserTime = System.currentTimeMillis() - start + taskStart = System.currentTimeMillis() // Run it val result: Any = deserializedTask.run(taskId) @@ -174,13 +178,15 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: logInfo("Finished " + taskId) deserializedTask.metrics.get.executorRunTime = serviceTime.toInt deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt - val taskResult = new TaskResult(result, accumUpdates, deserializedTask.metrics.getOrElse(null)) val serializedResult = ser.serialize(taskResult) localActor ! LocalStatusUpdate(taskId, TaskState.FINISHED, serializedResult) } catch { case t: Throwable => { - val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) + val serviceTime = System.currentTimeMillis() - taskStart + val metrics = attemptedTask.flatMap(t => t.metrics) + metrics.foreach{m => m.executorRunTime = serviceTime.toInt} + val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) localActor ! LocalStatusUpdate(taskId, TaskState.FAILED, ser.serialize(failure)) } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index 499116f653..f12fec41d5 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -152,7 +152,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas info.markFailed() decreaseRunningTasks(1) val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) - sched.listener.taskEnded(task, reason, null, null, info, null) + sched.listener.taskEnded(task, reason, null, null, info, reason.metrics.getOrElse(null)) if (!finished(index)) { copiesRunning(index) -= 1 numFailures(index) += 1 diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 24cfe36aaa..8bbc6ce88e 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -41,7 +41,17 @@ private[spark] object UIWorkloadGenerator { 1 }.count }), - ("Job with delays", baseData.map(x => Thread.sleep(1000)).count) + ("Partially failed phase (longer tasks)", { + baseData.map{x => + val probFailure = (4.0 / NUM_PARTITIONS) + if (nextFloat() < probFailure) { + Thread.sleep(100) + throw new Exception("This is a task failure") + } + 1 + }.count + }), + ("Job with delays", baseData.map(x => Thread.sleep(100)).count) ) while (true) { diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index aafa414055..36b1cd00ed 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -78,18 +78,18 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - val failureInfo: Option[ExceptionFailure] = + val (failureInfo, metrics): (Option[ExceptionFailure], TaskMetrics) = taskEnd.reason match { case e: ExceptionFailure => stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - Some(e) + (Some(e), e.metrics.get) case _ => stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - None + (None, taskEnd.taskMetrics) } val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]()) - taskList += ((taskEnd.taskInfo, taskEnd.taskMetrics, failureInfo)) + taskList += ((taskEnd.taskInfo, metrics, failureInfo)) stageToTaskInfos(sid) = taskList } From 44a2440039a35784b7dfed2e36b96096c3424d33 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Sun, 7 Jul 2013 01:33:09 +0800 Subject: [PATCH 139/419] Remove active job from idToActiveJob when job finished or aborted --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 51b10ed045..cbd375e5c1 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -524,6 +524,7 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { + idToActiveJob -= stage.priority activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) @@ -671,6 +672,7 @@ class DAGScheduler( val error = new SparkException("Job failed: " + reason) job.listener.jobFailed(error) sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) + idToActiveJob -= resultStage.priority activeJobs -= job resultStageToJob -= resultStage } From 32b9d21a97d1c93f174551000d06cc429f317827 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 6 Jul 2013 16:36:13 -0700 Subject: [PATCH 140/419] Fix occasional failure in UI listener. If a task fails before the metrics are initialized, it remains possible that the metrics field will be `None`. This patch accounts for that possbility by keeping metrics as an `Option` at all times. --- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 14 +++++++------- .../src/main/scala/spark/ui/jobs/StagePage.scala | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 36b1cd00ed..84730cc091 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -51,7 +51,7 @@ private[spark] class JobProgressListener extends SparkListener { val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = - HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]]() + HashMap[Int, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() override def onJobStart(jobStart: SparkListenerJobStart) {} @@ -78,17 +78,17 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - val (failureInfo, metrics): (Option[ExceptionFailure], TaskMetrics) = + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - (Some(e), e.metrics.get) + (Some(e), e.metrics) case _ => stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - (None, taskEnd.taskMetrics) + (None, Some(taskEnd.taskMetrics)) } val taskList = stageToTaskInfos.getOrElse( - sid, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]()) + sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskEnd.taskInfo, metrics, failureInfo)) stageToTaskInfos(sid) = taskList } @@ -111,7 +111,7 @@ private[spark] class JobProgressListener extends SparkListener { def hasShuffleRead(stageID: Int): Boolean = { // This is written in a slightly complicated way to avoid having to scan all tasks for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.shuffleReadMetrics.isDefined + if (s._2 != null) return s._2.flatMap(m => m.shuffleReadMetrics).isDefined } return false // No tasks have finished for this stage } @@ -120,7 +120,7 @@ private[spark] class JobProgressListener extends SparkListener { def hasShuffleWrite(stageID: Int): Boolean = { // This is written in a slightly complicated way to avoid having to scan all tasks for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.shuffleWriteMetrics.isDefined + if (s._2 != null) return s._2.flatMap(m => m.shuffleWriteMetrics).isDefined } return false // No tasks have finished for this stage } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 49e84880cf..51b82b6a8c 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -52,7 +52,7 @@ private[spark] class StagePage(parent: JobProgressUI) { } else { val serviceTimes = validTasks.map{case (info, metrics, exception) => - metrics.executorRunTime.toDouble} + metrics.get.executorRunTime.toDouble} val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( ms => parent.formatDuration(ms.toLong)) @@ -61,13 +61,13 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleReadSizes = validTasks.map { case(info, metrics, exception) => - metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) val shuffleWriteSizes = validTasks.map { case(info, metrics, exception) => - metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble } val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) @@ -87,21 +87,21 @@ private[spark] class StagePage(parent: JobProgressUI) { } - def taskRow(taskData: (TaskInfo, TaskMetrics, Option[ExceptionFailure])): Seq[Node] = { + def taskRow(taskData: (TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])): Seq[Node] = { def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = trace.map(e => {e.toString}) val (info, metrics, exception) = taskData {info.taskId} - m.executorRunTime.toString}.getOrElse("1")}> - {Option(metrics).map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} + m.executorRunTime.toString}.getOrElse("1")}> + {metrics.map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} - {Option(metrics).flatMap{m => m.shuffleReadMetrics}.map{s => + {metrics.flatMap{m => m.shuffleReadMetrics}.map{s => {Utils.memoryBytesToString(s.remoteBytesRead)}}.getOrElse("")} - {Option(metrics).flatMap{m => m.shuffleWriteMetrics}.map{s => + {metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => {Utils.memoryBytesToString(s.shuffleBytesWritten)}}.getOrElse("")} {exception.map(e => From fd6665122b4bd3aa2d56ec2f6c6028e1b8a18b7f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 6 Jul 2013 16:45:15 -0700 Subject: [PATCH 141/419] Fix some other references to Cloudera Avro and updated Avro version --- pom.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 3184fd376a..017c242464 100644 --- a/pom.xml +++ b/pom.xml @@ -512,12 +512,12 @@ org.apache.avro avro - 1.7.1.cloudera.2 + 1.7.4 org.apache.avro avro-ipc - 1.7.1.cloudera.2 + 1.7.4 org.jboss.netty @@ -579,12 +579,12 @@ org.apache.avro avro - 1.7.1 + 1.7.4 org.apache.avro avro-ipc - 1.7.1 + 1.7.4 From f78f8d0b416ef4d88883d8f32382661f4c2ac52d Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Sat, 6 Jul 2013 16:46:53 -0700 Subject: [PATCH 142/419] fix formatting and use Vector instead of List to maintain order --- .../mllib/regression/RidgeRegression.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 8343f28139..36cda721dd 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -15,7 +15,7 @@ class RidgeRegressionModel( val weights: DoubleMatrix, val intercept: Double, val lambdaOpt: Double, - val lambdas: List[(Double, Double, DoubleMatrix)]) + val lambdas: Seq[(Double, Double, DoubleMatrix)]) extends RegressionModel { override def predict(testData: RDD[Array[Double]]): RDD[Double] = { @@ -99,12 +99,10 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) } // Binary search for the best assignment to lambda. - def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { + def binSearch(low: Double, high: Double): Seq[(Double, Double, DoubleMatrix)] = { @tailrec - def loop( - low: Double, - high: Double, - acc: List[(Double, Double, DoubleMatrix)]): List[(Double, Double, DoubleMatrix)] = { + def loop(low: Double, high: Double, acc: Seq[(Double, Double, DoubleMatrix)]) + : Seq[(Double, Double, DoubleMatrix)] = { val mid = (high - low) / 2 + low val lowValue = crossValidate((mid - low) / 2 + low) val highValue = crossValidate((high - mid) / 2 + mid) @@ -114,14 +112,13 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) (mid - (high-low)/4, high) } if (newHigh - newLow > 1.0E-7) { - // :: is list prepend in Scala. - loop(newLow, newHigh, lowValue :: highValue :: acc) + loop(newLow, newHigh, acc :+ lowValue :+ highValue) } else { - lowValue :: highValue :: acc + acc :+ lowValue :+ highValue } } - loop(low, high, Nil) + loop(low, high, Vector.empty) } // Actually compute the best lambda @@ -143,6 +140,7 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) model } } + /** * Top-level methods for calling Ridge Regression. */ From a948f0672541bd68be020f07134659aee2f38403 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 02:30:45 -0700 Subject: [PATCH 143/419] Suppress log messages in sbt test with two changes: 1. Set akka log level to ERROR before shutting down the actorSystem. This avoids akka log messages (like Spray) from falling back to INFO on the Stdout logger 2. Initialize netty to use SLF4J in LocalSparkContext. This ensures that stack trace thrown during shutdown is handled by SLF4J instead of stdout --- core/src/main/scala/spark/SparkEnv.scala | 2 ++ .../main/scala/spark/deploy/LocalSparkCluster.scala | 4 ++++ core/src/test/scala/spark/FileServerSuite.scala | 1 - core/src/test/scala/spark/LocalSparkContext.scala | 11 ++++++++++- 4 files changed, 16 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index ec59b4f48f..16b00d15aa 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -5,6 +5,7 @@ import serializer.Serializer import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider +import akka.event.{Logging => AkkaLogging} import spark.broadcast.BroadcastManager import spark.storage.BlockManager @@ -51,6 +52,7 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() + actorSystem.eventStream.setLogLevel(AkkaLogging.ErrorLevel) actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 55bb61b0cc..cb85419ae4 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -1,6 +1,7 @@ package spark.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.event.{Logging => AkkaLogging} import spark.deploy.worker.Worker import spark.deploy.master.Master @@ -43,8 +44,11 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I def stop() { logInfo("Shutting down local Spark cluster.") // Stop the workers before the master so they don't get upset that it disconnected + workerActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) workerActorSystems.foreach(_.shutdown()) workerActorSystems.foreach(_.awaitTermination()) + + masterActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) masterActorSystems.foreach(_.shutdown()) masterActorSystems.foreach(_.awaitTermination()) } diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index f1a35bced3..9c24ca430d 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -85,7 +85,6 @@ class FileServerSuite extends FunSuite with LocalSparkContext { in.close() _ * fileVal + _ * fileVal }.collect - println(result) assert(result.toSet === Set((1,200), (2,300), (3,500))) } diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala index 76d5258b02..bd184222ed 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -2,12 +2,21 @@ package spark import org.scalatest.Suite import org.scalatest.BeforeAndAfterEach +import org.scalatest.BeforeAndAfterAll + +import org.jboss.netty.logging.InternalLoggerFactory +import org.jboss.netty.logging.Slf4JLoggerFactory /** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */ -trait LocalSparkContext extends BeforeAndAfterEach { self: Suite => +trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ + override def beforeAll() { + InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory()); + super.beforeAll() + } + override def afterEach() { resetSparkContext() super.afterEach() From 7d6d9e6ab226579518f1c7fbe108c4e66acc6ed0 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 02:45:55 -0700 Subject: [PATCH 144/419] Set DriverSuite log level to WARN --- core/src/test/scala/spark/DriverSuite.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index 5e84b3a66a..31c3dd75fb 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -2,6 +2,9 @@ package spark import java.io.File +import org.apache.log4j.Logger +import org.apache.log4j.Level + import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.prop.TableDrivenPropertyChecks._ @@ -27,6 +30,7 @@ class DriverSuite extends FunSuite with Timeouts { */ object DriverWithoutCleanup { def main(args: Array[String]) { + Logger.getRootLogger().setLevel(Level.WARN) val sc = new SparkContext(args(0), "DriverWithoutCleanup") sc.parallelize(1 to 100, 4).count() } From 3350ad0d7fc3ecece78f87d7aa6a727e48b21c8c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 03:32:26 -0700 Subject: [PATCH 145/419] Catch RejectedExecution exception in Checkpoint handler. --- .../src/main/scala/spark/streaming/Checkpoint.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 66e67cbfa1..450e48d66e 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -8,7 +8,7 @@ import org.apache.hadoop.conf.Configuration import java.io._ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import java.util.concurrent.Executors - +import java.util.concurrent.RejectedExecutionException private[streaming] class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) @@ -91,7 +91,12 @@ class CheckpointWriter(checkpointDir: String) extends Logging { oos.writeObject(checkpoint) oos.close() bos.close() - executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + try { + executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + } catch { + case rej: RejectedExecutionException => + logError("Could not submit checkpoint task to the thread pool executor", rej) + } } def stop() { From d362d0f4117268bdef265041ff291700ddd49b43 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 03:33:32 -0700 Subject: [PATCH 146/419] Ignore stderr when calling cat on a non-existing file --- core/src/test/scala/spark/PipedRDDSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index 1c9ca50811..d263bb00e9 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -67,7 +67,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { test("pipe with non-zero exit status") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe("cat nonexistent_file") + val piped = nums.pipe(Seq("cat nonexistent_file", "2>", "/dev/null")) intercept[SparkException] { piped.collect() } From 4af0d63cb14db902cbd1dbdeeb68f1fcec4b2e97 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 10:42:43 -0700 Subject: [PATCH 147/419] Remove akka LogLevel fix as we no longer use spray --- core/src/main/scala/spark/SparkEnv.scala | 2 -- core/src/main/scala/spark/deploy/LocalSparkCluster.scala | 3 --- 2 files changed, 5 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 16b00d15aa..ec59b4f48f 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -5,7 +5,6 @@ import serializer.Serializer import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider -import akka.event.{Logging => AkkaLogging} import spark.broadcast.BroadcastManager import spark.storage.BlockManager @@ -52,7 +51,6 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() - actorSystem.eventStream.setLogLevel(AkkaLogging.ErrorLevel) actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index cb85419ae4..939f26b6f4 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -1,7 +1,6 @@ package spark.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import akka.event.{Logging => AkkaLogging} import spark.deploy.worker.Worker import spark.deploy.master.Master @@ -44,11 +43,9 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I def stop() { logInfo("Shutting down local Spark cluster.") // Stop the workers before the master so they don't get upset that it disconnected - workerActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) workerActorSystems.foreach(_.shutdown()) workerActorSystems.foreach(_.awaitTermination()) - masterActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) masterActorSystems.foreach(_.shutdown()) masterActorSystems.foreach(_.awaitTermination()) } From be123aa6ef90480ad61663eed6e8ea479b047fad Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Sun, 7 Jul 2013 15:35:06 -0700 Subject: [PATCH 148/419] update to use ListBuffer, faster than Vector for append operations --- .../spark/mllib/regression/RidgeRegression.scala | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 36cda721dd..f66025bc0b 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -7,6 +7,7 @@ import org.jblas.DoubleMatrix import org.jblas.Solve import scala.annotation.tailrec +import scala.collection.mutable /** * Ridge Regression from Joseph Gonzalez's implementation in MLBase @@ -100,9 +101,10 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) // Binary search for the best assignment to lambda. def binSearch(low: Double, high: Double): Seq[(Double, Double, DoubleMatrix)] = { + val buffer = mutable.ListBuffer.empty[(Double, Double, DoubleMatrix)] + @tailrec - def loop(low: Double, high: Double, acc: Seq[(Double, Double, DoubleMatrix)]) - : Seq[(Double, Double, DoubleMatrix)] = { + def loop(low: Double, high: Double): Seq[(Double, Double, DoubleMatrix)] = { val mid = (high - low) / 2 + low val lowValue = crossValidate((mid - low) / 2 + low) val highValue = crossValidate((high - mid) / 2 + mid) @@ -112,13 +114,15 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) (mid - (high-low)/4, high) } if (newHigh - newLow > 1.0E-7) { - loop(newLow, newHigh, acc :+ lowValue :+ highValue) + buffer += lowValue += highValue + loop(newLow, newHigh) } else { - acc :+ lowValue :+ highValue + buffer += lowValue += highValue + buffer.result() } } - loop(low, high, Vector.empty) + loop(low, high) } // Actually compute the best lambda From 8c1d1c98e0cea8d2b20ac10b84aa76d8e22a1661 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Mon, 8 Jul 2013 12:25:46 -0700 Subject: [PATCH 149/419] Explicitly set class loader for MesosSchedulerDriver callbacks. --- .../scheduler/mesos/MesosSchedulerBackend.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index ca7fab4cc5..e73b780fcb 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -43,8 +43,12 @@ private[spark] class MesosSchedulerBackend( // An ExecutorInfo for our tasks var execArgs: Array[Byte] = null + var classLoader: ClassLoader = null + override def start() { synchronized { + classLoader = Thread.currentThread.getContextClassLoader + new Thread("MesosSchedulerBackend driver") { setDaemon(true) override def run() { @@ -114,9 +118,16 @@ private[spark] class MesosSchedulerBackend( return execArgs } + private def setClassLoader() { + // Since native code starts the thread our callbacks run in, it may not correctly + // inherit and custom class loaders. Therefore, set the class loader manually. + Thread.currentThread.setContextClassLoader(classLoader) + } + override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { + setClassLoader() logInfo("Registered as framework ID " + frameworkId.getValue) registeredLock.synchronized { isRegistered = true @@ -142,6 +153,7 @@ private[spark] class MesosSchedulerBackend( * tasks are balanced across the cluster. */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { + setClassLoader() synchronized { // Build a big list of the offerable workers, and remember their indices so that we can // figure out which Offer to reply to for each worker @@ -224,6 +236,7 @@ private[spark] class MesosSchedulerBackend( } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + setClassLoader() val tid = status.getTaskId.getValue.toLong val state = TaskState.fromMesos(status.getState) synchronized { From bf4c9a5e0fca2dfc960120a7f3c5fab0b87e3850 Mon Sep 17 00:00:00 2001 From: Ameet Talwalkar Date: Mon, 8 Jul 2013 14:37:42 -0700 Subject: [PATCH 150/419] renamed with labeled prefix --- .../scala/spark/mllib/regression/LogisticRegression.scala | 2 +- .../spark/mllib/regression/LogisticRegressionGenerator.scala | 2 +- .../main/scala/spark/mllib/regression/RidgeRegression.scala | 2 +- .../spark/mllib/regression/RidgeRegressionGenerator.scala | 2 +- mllib/src/main/scala/spark/mllib/util/MLUtils.scala | 4 ++-- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index 448ab9dce9..e4db7bb9b7 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -150,7 +150,7 @@ object LogisticRegression { System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") - val data = MLUtils.loadData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)) val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) sc.stop() diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala index 9f6abab70b..6e7c023bac 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala @@ -35,7 +35,7 @@ object LogisticRegressionGenerator { (y, x) } - MLUtils.saveData(data, outputPath) + MLUtils.saveLabeledData(data, outputPath) sc.stop() } } diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index f66025bc0b..5f813df402 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -187,7 +187,7 @@ object RidgeRegression { System.exit(1) } val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)) val model = RidgeRegression.train(data, 0, 1000) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala index c9ac4a8b07..b83f505d8e 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala @@ -49,7 +49,7 @@ object RidgeRegressionGenerator { } } - MLUtils.saveData(data, outputPath) + MLUtils.saveLabeledData(data, outputPath) sc.stop() } } diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala index 0a4a037c71..08a031dded 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -19,7 +19,7 @@ object MLUtils { * @return An RDD of tuples. For each tuple, the first element is the label, and the second * element represents the feature values (an array of Double). */ - def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { + def loadLabeledData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { sc.textFile(dir).map { line => val parts = line.split(",") val label = parts(0).toDouble @@ -28,7 +28,7 @@ object MLUtils { } } - def saveData(data: RDD[(Double, Array[Double])], dir: String) { + def saveLabeledData(data: RDD[(Double, Array[Double])], dir: String) { val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) dataStr.saveAsTextFile(dir) } From afdaf430bd8d7da3ee3323cf03b00d4214159626 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Jul 2013 15:38:29 -0700 Subject: [PATCH 151/419] Explicit dependencies for scala-library and scalap to prevent 2.9.2 vs. 2.9.3 problems --- core/pom.xml | 10 ++++++++++ examples/pom.xml | 5 +++++ streaming/pom.xml | 5 +++++ 3 files changed, 20 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 5edafb3706..39dba46d7a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -55,6 +55,16 @@ com.typesafe.akka akka-slf4j + + org.scala-lang + scalap + 2.9.3 + + + org.scala-lang + scala-library + 2.9.3 + net.liftweb lift-json_2.9.2 diff --git a/examples/pom.xml b/examples/pom.xml index 78ec58729b..7d9769e8e4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -15,6 +15,11 @@ http://spark-project.org/ + + org.scala-lang + scala-library + 2.9.3 + org.eclipse.jetty jetty-server diff --git a/streaming/pom.xml b/streaming/pom.xml index 4dc9a19d51..07725f9484 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -58,6 +58,11 @@ twitter4j-stream 3.0.3 + + org.scala-lang + scala-library + 2.9.3 + com.typesafe.akka akka-zeromq From 0b39d66f3f815f2d406d83a41db0f46d097baaa7 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Jul 2013 16:07:09 -0700 Subject: [PATCH 152/419] pom cleanup --- core/pom.xml | 2 -- examples/pom.xml | 1 - pom.xml | 10 ++++++++++ streaming/pom.xml | 1 - 4 files changed, 10 insertions(+), 4 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 39dba46d7a..dbb2da9a9c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -58,12 +58,10 @@ org.scala-lang scalap - 2.9.3 org.scala-lang scala-library - 2.9.3 net.liftweb diff --git a/examples/pom.xml b/examples/pom.xml index 7d9769e8e4..1976765c3d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -18,7 +18,6 @@ org.scala-lang scala-library - 2.9.3 org.eclipse.jetty diff --git a/pom.xml b/pom.xml index 017c242464..63dd80b5fe 100644 --- a/pom.xml +++ b/pom.xml @@ -241,6 +241,16 @@ jline ${scala.version} + + org.scala-lang + scala-library + ${scala.version} + + + org.scala-lang + scalap + ${scala.version} + log4j diff --git a/streaming/pom.xml b/streaming/pom.xml index 07725f9484..2fb5bbdeb5 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -61,7 +61,6 @@ org.scala-lang scala-library - 2.9.3 com.typesafe.akka From c1d44be80580f0fad6bb1805bbcf74a34f536d8c Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Tue, 9 Jul 2013 15:18:28 +0800 Subject: [PATCH 153/419] Bug fix: SPARK-796 --- .../spark/deploy/yarn/ApplicationMaster.scala | 49 ++++++++++++------- 1 file changed, 32 insertions(+), 17 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index f19648ec68..9bc692d480 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -27,6 +27,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var yarnAllocator: YarnAllocationHandler = null + private var isFinished:Boolean = false def run() { @@ -68,10 +69,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Wait for the user class to Finish userThread.join() - - // Finish the ApplicationMaster - finishApplicationMaster() - // TODO: Exit based on success/failure + System.exit(0) } @@ -131,10 +129,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e .getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { - // Copy - var mainArgs: Array[String] = new Array[String](args.userArgs.size()) - args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) - mainMethod.invoke(null, mainArgs) + try{ + // Copy + var mainArgs: Array[String] = new Array[String](args.userArgs.size()) + args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) + mainMethod.invoke(null, mainArgs) + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + } catch { + case th: Throwable => + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) + logError("Finish ApplicationMaster with ",th) + } } } t.start() @@ -235,14 +240,22 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } */ - - def finishApplicationMaster() { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - // TODO: Check if the application has failed or succeeded - finishReq.setFinishApplicationStatus(FinalApplicationStatus.SUCCEEDED) - resourceManager.finishApplicationMaster(finishReq) + + def finishApplicationMaster(status: FinalApplicationStatus) { + + synchronized { + if(isFinished){ + return + } + isFinished = true + + logInfo("finishApplicationMaster with "+status) + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + resourceManager.finishApplicationMaster(finishReq) + } } } @@ -291,7 +304,9 @@ object ApplicationMaster { logInfo("Invoking sc stop from shutdown hook") sc.stop() // best case ... - for (master <- applicationMasters) master.finishApplicationMaster + for (master <- applicationMasters) { + master.finishApplicationMaster(FinalApplicationStatus.KILLED) + } } } ) } From e47253e0cca1359b49f113dbf258c2c204e3bfc1 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 9 Jul 2013 01:13:12 -0700 Subject: [PATCH 154/419] Reset ClassLoader in MesosSchedulerBackend, too. (per review comments). Also set ClassLoader for all mesos callbacks, not just statusUpdate, registered. --- .../mesos/MesosSchedulerBackend.scala | 148 ++++++++++-------- 1 file changed, 87 insertions(+), 61 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index e73b780fcb..e83368b98d 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -118,20 +118,28 @@ private[spark] class MesosSchedulerBackend( return execArgs } - private def setClassLoader() { - // Since native code starts the thread our callbacks run in, it may not correctly - // inherit and custom class loaders. Therefore, set the class loader manually. + private def setClassLoader(): ClassLoader = { + val oldClassLoader = Thread.currentThread.getContextClassLoader Thread.currentThread.setContextClassLoader(classLoader) + return oldClassLoader + } + + private def restoreClassLoader(oldClassLoader: ClassLoader) { + Thread.currentThread.setContextClassLoader(oldClassLoader) } override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - setClassLoader() - logInfo("Registered as framework ID " + frameworkId.getValue) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() + val oldClassLoader = setClassLoader() + try { + logInfo("Registered as framework ID " + frameworkId.getValue) + registeredLock.synchronized { + isRegistered = true + registeredLock.notifyAll() + } + } finally { + restoreClassLoader(oldClassLoader) } } @@ -153,50 +161,54 @@ private[spark] class MesosSchedulerBackend( * tasks are balanced across the cluster. */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { - setClassLoader() - synchronized { - // Build a big list of the offerable workers, and remember their indices so that we can - // figure out which Offer to reply to for each worker - val offerableIndices = new ArrayBuffer[Int] - val offerableWorkers = new ArrayBuffer[WorkerOffer] + val oldClassLoader = setClassLoader() + try { + synchronized { + // Build a big list of the offerable workers, and remember their indices so that we can + // figure out which Offer to reply to for each worker + val offerableIndices = new ArrayBuffer[Int] + val offerableWorkers = new ArrayBuffer[WorkerOffer] - def enoughMemory(o: Offer) = { - val mem = getResource(o.getResourcesList, "mem") - val slaveId = o.getSlaveId.getValue - mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId) - } + def enoughMemory(o: Offer) = { + val mem = getResource(o.getResourcesList, "mem") + val slaveId = o.getSlaveId.getValue + mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId) + } - for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { - offerableIndices += index - offerableWorkers += new WorkerOffer( - offer.getSlaveId.getValue, - offer.getHostname, - getResource(offer.getResourcesList, "cpus").toInt) - } + for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { + offerableIndices += index + offerableWorkers += new WorkerOffer( + offer.getSlaveId.getValue, + offer.getHostname, + getResource(offer.getResourcesList, "cpus").toInt) + } - // Call into the ClusterScheduler - val taskLists = scheduler.resourceOffers(offerableWorkers) + // Call into the ClusterScheduler + val taskLists = scheduler.resourceOffers(offerableWorkers) - // Build a list of Mesos tasks for each slave - val mesosTasks = offers.map(o => Collections.emptyList[MesosTaskInfo]()) - for ((taskList, index) <- taskLists.zipWithIndex) { - if (!taskList.isEmpty) { - val offerNum = offerableIndices(index) - val slaveId = offers(offerNum).getSlaveId.getValue - slaveIdsWithExecutors += slaveId - mesosTasks(offerNum) = new JArrayList[MesosTaskInfo](taskList.size) - for (taskDesc <- taskList) { - taskIdToSlaveId(taskDesc.taskId) = slaveId - mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId)) + // Build a list of Mesos tasks for each slave + val mesosTasks = offers.map(o => Collections.emptyList[MesosTaskInfo]()) + for ((taskList, index) <- taskLists.zipWithIndex) { + if (!taskList.isEmpty) { + val offerNum = offerableIndices(index) + val slaveId = offers(offerNum).getSlaveId.getValue + slaveIdsWithExecutors += slaveId + mesosTasks(offerNum) = new JArrayList[MesosTaskInfo](taskList.size) + for (taskDesc <- taskList) { + taskIdToSlaveId(taskDesc.taskId) = slaveId + mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId)) + } } } - } - // Reply to the offers - val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? - for (i <- 0 until offers.size) { - d.launchTasks(offers(i).getId, mesosTasks(i), filters) + // Reply to the offers + val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? + for (i <- 0 until offers.size) { + d.launchTasks(offers(i).getId, mesosTasks(i), filters) + } } + } finally { + restoreClassLoader(oldClassLoader) } } @@ -236,24 +248,33 @@ private[spark] class MesosSchedulerBackend( } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { - setClassLoader() - val tid = status.getTaskId.getValue.toLong - val state = TaskState.fromMesos(status.getState) - synchronized { - if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { - // We lost the executor on this slave, so remember that it's gone - slaveIdsWithExecutors -= taskIdToSlaveId(tid) - } - if (isFinished(status.getState)) { - taskIdToSlaveId.remove(tid) + val oldClassLoader = setClassLoader() + try { + val tid = status.getTaskId.getValue.toLong + val state = TaskState.fromMesos(status.getState) + synchronized { + if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { + // We lost the executor on this slave, so remember that it's gone + slaveIdsWithExecutors -= taskIdToSlaveId(tid) + } + if (isFinished(status.getState)) { + taskIdToSlaveId.remove(tid) + } } + scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) + } finally { + restoreClassLoader(oldClassLoader) } - scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) } override def error(d: SchedulerDriver, message: String) { - logError("Mesos error: " + message) - scheduler.error(message) + val oldClassLoader = setClassLoader() + try { + logError("Mesos error: " + message) + scheduler.error(message) + } finally { + restoreClassLoader(oldClassLoader) + } } override def stop() { @@ -269,11 +290,16 @@ private[spark] class MesosSchedulerBackend( override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { - logInfo("Mesos slave lost: " + slaveId.getValue) - synchronized { - slaveIdsWithExecutors -= slaveId.getValue + val oldClassLoader = setClassLoader() + try { + logInfo("Mesos slave lost: " + slaveId.getValue) + synchronized { + slaveIdsWithExecutors -= slaveId.getValue + } + scheduler.executorLost(slaveId.getValue, reason) + } finally { + restoreClassLoader(oldClassLoader) } - scheduler.executorLost(slaveId.getValue, reason) } override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { From aaa7b081df760a29ce5cdcd51d6b71422cba68d5 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Tue, 9 Jul 2013 20:03:01 +0800 Subject: [PATCH 155/419] according to mridulm's comments to adjust the code --- .../spark/deploy/yarn/ApplicationMaster.scala | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 9bc692d480..776db201f9 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -122,23 +122,26 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } } - + private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader) .getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { + var successed = false try{ // Copy var mainArgs: Array[String] = new Array[String](args.userArgs.size()) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) mainMethod.invoke(null, mainArgs) - ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - } catch { - case th: Throwable => + successed = true + } finally { + if(successed){ + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + }else{ ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) - logError("Finish ApplicationMaster with ",th) + } } } } @@ -248,14 +251,15 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e return } isFinished = true - - logInfo("finishApplicationMaster with "+status) - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - resourceManager.finishApplicationMaster(finishReq) } + + logInfo("finishApplicationMaster with " + status) + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + resourceManager.finishApplicationMaster(finishReq) + } } @@ -304,6 +308,9 @@ object ApplicationMaster { logInfo("Invoking sc stop from shutdown hook") sc.stop() // best case ... + // due to the sparkContext is stopped and ApplicationMaster is down, + // the status of registered masters should be set KILLED better than FAILED. + // need discussion for (master <- applicationMasters) { master.finishApplicationMaster(FinalApplicationStatus.KILLED) } From 13fc6f248c7231501d8c53ad13641fa996e06be4 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 9 Jul 2013 13:42:51 -0700 Subject: [PATCH 156/419] Clean commit of log paging --- .../deploy/master/ui/ApplicationPage.scala | 8 ++- .../spark/deploy/worker/ui/IndexPage.scala | 8 ++- .../spark/deploy/worker/ui/WorkerWebUI.scala | 61 +++++++++++++++++++ 3 files changed, 75 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 33a16b5d84..ea88421532 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -38,7 +38,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { state.completedApps.find(_.id == appId).getOrElse(null) }) - val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs", "Log Pages") val executors = app.executors.values.toSeq val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) @@ -95,6 +95,12 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { stderr + + stdout + stderr + } } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index e466129c1a..7cf98b473e 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -29,7 +29,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs", "Log Pages") val runningExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) val finishedExecutorTable = @@ -93,6 +93,12 @@ private[spark] class IndexPage(parent: WorkerWebUI) { stderr + + stdout-page + stderr-page + } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 16564d5619..5b0c785b00 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -13,6 +13,10 @@ import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ +import scala.xml._ +import spark.ui.UIUtils +import scala.io.Source._ + /** * Web UI server for the standalone worker. */ @@ -33,6 +37,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val handlers = Array[(String, Handler)]( ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), ("/log", (request: HttpServletRequest) => log(request)), + ("/logPage", (request: HttpServletRequest) => logPage(request)), ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), ("*", (request: HttpServletRequest) => indexPage.render(request)) ) @@ -65,6 +70,62 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option pre + Utils.lastNBytes(path, math.min(numBytes, maxBytes)) } + def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { + val appId = request.getParameter("appId") + val executorId = request.getParameter("executorId") + val logType = request.getParameter("logType") + val getOffset = request.getParameter("offset") + val getLineLength = request.getParameter("lineLength") + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) + val source = fromFile(path) + val lines = source.getLines().toArray + val logLength = lines.length + val offset = { + if (getOffset == null) 0 + else if (getOffset.toInt < 0) 0 + else getOffset.toInt + } + val lineLength = { + if (getLineLength == null) 0 + else getLineLength.toInt + } + val logText = "" + lines.slice(offset, offset+lineLength).mkString("\n") + "" + val logXML = XML.loadString(logText) + val backButton = + if (offset > 0) { + if (offset-lineLength < 0) { + + } + else { + + } + } + else { + + } + val nextButton = + if (offset+lineLength < logLength) { + + } + else { + + } + val content = + + +
      + {backButton} + {nextButton} +

      +
      {logXML}
      + {backButton} + {nextButton} + + + source.close() + UIUtils.basicSparkPage(content, "Log Page for " + appId) + } + def stop() { server.foreach(_.stop()) } From b6072b58bf795093df0e5f8424413fcd0cce6323 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 9 Jul 2013 17:25:10 -0700 Subject: [PATCH 157/419] Fixes style, makes "std__-page" consistent, reads only parts of files --- core/src/main/scala/spark/Utils.scala | 14 +++++ .../deploy/master/ui/ApplicationPage.scala | 8 +-- .../spark/deploy/worker/ui/IndexPage.scala | 4 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 54 +++++++++---------- 4 files changed, 45 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 64547bbdcd..dfc30469cf 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -635,6 +635,20 @@ private object Utils extends Logging { Source.fromBytes(buff).mkString } + /** Return an array containing part of a file from byte 'a' to 'b'. */ + def offsetBytes(path: String, a: Int, b: Int): String = { + val file = new File(path) + val length = file.length() + val buff = new Array[Byte](math.min(b-a, length.toInt)) + val skip = math.max(0, a) + val stream = new FileInputStream(file) + + stream.skip(skip) + stream.read(buff) + stream.close() + Source.fromBytes(buff).mkString + } + /** * Clone an object using a Spark serializer. */ diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index ea88421532..cc32728c1c 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -96,10 +96,10 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stderr - stdout - stderr + stdout-page + stderr-page } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 7cf98b473e..d532aa9e95 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -94,9 +94,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { .format(executor.appId, executor.execId)}>stderr - stdout-page - stderr-page diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 5b0c785b00..b018f80c85 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -9,13 +9,13 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} +import scala.io.Source._ +import scala.xml._ + import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ - -import scala.xml._ import spark.ui.UIUtils -import scala.io.Source._ /** * Web UI server for the standalone worker. @@ -74,42 +74,39 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val getOffset = request.getParameter("offset") - val getLineLength = request.getParameter("lineLength") + val offset = Option(request.getParameter("offset")).map(_.toInt).getOrElse(0) + + val maxBytes = 1024 * 1024 + val defaultBytes = 100 * 1024 + val byteLength = Option(request.getParameter("byteLength")).flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val source = fromFile(path) - val lines = source.getLines().toArray - val logLength = lines.length - val offset = { - if (getOffset == null) 0 - else if (getOffset.toInt < 0) 0 - else getOffset.toInt - } - val lineLength = { - if (getLineLength == null) 0 - else getLineLength.toInt - } - val logText = "" + lines.slice(offset, offset+lineLength).mkString("\n") + "" - val logXML = XML.loadString(logText) + val logLength = new File(path).length() + val logPageLength = math.min(byteLength, maxBytes) + val logText = {Utils.offsetBytes(path, offset, offset+logPageLength)} + val backButton = if (offset > 0) { - if (offset-lineLength < 0) { - - } - else { - - } + + + } else { } + val nextButton = - if (offset+lineLength < logLength) { - + if (offset+logPageLength < logLength) { + + + } else { } + val content = @@ -117,12 +114,11 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option {backButton} {nextButton}

      -
      {logXML}
      +
      {logText}
      {backButton} {nextButton} - source.close() UIUtils.basicSparkPage(content, "Log Page for " + appId) } From ce18b50d5ff37dc5c558d0602321a61887dd8b48 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Wed, 10 Jul 2013 19:11:43 +0800 Subject: [PATCH 158/419] set SUCCEEDED for all master in shutdown hook --- .../scala/spark/deploy/yarn/ApplicationMaster.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 776db201f9..68bb36d316 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -135,6 +135,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e var mainArgs: Array[String] = new Array[String](args.userArgs.size()) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) mainMethod.invoke(null, mainArgs) + // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR + // userThread will stop here unless it has uncaught exception thrown out + // It need shutdown hook to set SUCCEEDED successed = true } finally { if(successed){ @@ -308,11 +311,8 @@ object ApplicationMaster { logInfo("Invoking sc stop from shutdown hook") sc.stop() // best case ... - // due to the sparkContext is stopped and ApplicationMaster is down, - // the status of registered masters should be set KILLED better than FAILED. - // need discussion for (master <- applicationMasters) { - master.finishApplicationMaster(FinalApplicationStatus.KILLED) + master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } } } ) From 620a6974c6603f1c0e5a7cea8f0387a5d18f2e5e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 10:20:53 -0700 Subject: [PATCH 159/419] Allows for larger files, refactors lastNBytes, removes old Log column, fixes imports, uses map --- core/src/main/scala/spark/Utils.scala | 21 +++++++------------ .../deploy/master/ui/ApplicationPage.scala | 12 +++-------- .../spark/deploy/worker/ui/IndexPage.scala | 12 +++-------- .../spark/deploy/worker/ui/WorkerWebUI.scala | 7 ++----- 4 files changed, 15 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index dfc30469cf..849edc13ee 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,12 +621,12 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } - /** Return a string containing the last `n` bytes of a file. */ - def lastNBytes(path: String, n: Int): String = { + /** Return a string containing part of a file from byte 'a' to 'b'. */ + def offsetBytes(path: String, a: Long, b: Long): String = { val file = new File(path) val length = file.length() - val buff = new Array[Byte](math.min(n, length.toInt)) - val skip = math.max(0, length - n) + val buff = new Array[Byte](math.min((b-a).toInt, length.toInt)) + val skip = math.max(0, a) val stream = new FileInputStream(file) stream.skip(skip) @@ -635,18 +635,11 @@ private object Utils extends Logging { Source.fromBytes(buff).mkString } - /** Return an array containing part of a file from byte 'a' to 'b'. */ - def offsetBytes(path: String, a: Int, b: Int): String = { + /** Return a string containing the last `n` bytes of a file. */ + def lastNBytes(path: String, n: Int): String = { val file = new File(path) val length = file.length() - val buff = new Array[Byte](math.min(b-a, length.toInt)) - val skip = math.max(0, a) - val stream = new FileInputStream(file) - - stream.skip(skip) - stream.read(buff) - stream.close() - Source.fromBytes(buff).mkString + offsetBytes(path, length-n, length) } /** diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index cc32728c1c..49ced0d320 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -38,7 +38,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { state.completedApps.find(_.id == appId).getOrElse(null) }) - val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Log Pages") val executors = app.executors.values.toSeq val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) @@ -89,17 +89,11 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.cores} {executor.memory} {executor.state} - - stdout - stderr - stdout-page + .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stdout stderr-page + .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stderr } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index d532aa9e95..af9943853f 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -29,7 +29,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Log Pages") val runningExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) val finishedExecutorTable = @@ -87,17 +87,11 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
    • User: {executor.appDesc.user}
    - - stdout - stderr - stdout-page + .format(executor.appId, executor.execId)}>stdout stderr-page + .format(executor.appId, executor.execId)}>stderr } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index b018f80c85..602881d5f1 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -9,9 +9,6 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import scala.io.Source._ -import scala.xml._ - import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ @@ -74,11 +71,11 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toInt).getOrElse(0) + val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0).asInstanceOf[Long] val maxBytes = 1024 * 1024 val defaultBytes = 100 * 1024 - val byteLength = Option(request.getParameter("byteLength")).flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val logLength = new File(path).length() From 24705d0f46ce536bf829660f4506dcffd9ff799a Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 10 Jul 2013 10:33:11 -0700 Subject: [PATCH 160/419] adding takeOrdered() to RDD --- core/src/main/scala/spark/RDD.scala | 12 ++++++++++++ core/src/test/scala/spark/RDDSuite.scala | 18 ++++++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 106fb2960f..af52040fa6 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -781,6 +781,18 @@ abstract class RDD[T: ClassManifest]( }.toArray } + /** + * Returns the top K elements from this RDD as defined by + * the specified implicit Ordering[T] and maintains the + * ordering. + * @param num the number of top elements to return + * @param ord the implicit ordering for T + * @return an array of top elements + */ + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { + top(num)(ord.reverse).sorted(ord) + } + /** * Save this RDD as a text file, using string representations of elements. */ diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index e41ae385c0..fe17d1d5e7 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -252,6 +252,24 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(topK.sorted === Array("b", "a")) } + test("takeOrdered with predefined ordering") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + val rdd = sc.makeRDD(nums, 2) + val sortedTopK = rdd.takeOrdered(5) + assert(sortedTopK.size === 5) + assert(sortedTopK === Array(1, 2, 3, 4, 5)) + } + + test("takeOrdered with custom ordering") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + implicit val ord = implicitly[Ordering[Int]].reverse + val rdd = sc.makeRDD(nums, 2) + val sortedTopK = rdd.takeOrdered(5) + assert(sortedTopK.size === 5) + assert(sortedTopK === Array(10, 9, 8, 7, 6)) + assert(sortedTopK === nums.sorted(ord).take(5)) + } + test("takeSample") { val data = sc.parallelize(1 to 100, 2) for (seed <- 1 to 5) { From ee4ce2fc51112387f28d7b422969ca2e9736e95f Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 10 Jul 2013 10:46:04 -0700 Subject: [PATCH 161/419] adding takeOrdered to java API --- .../scala/spark/api/java/JavaRDDLike.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index b555f2030a..94b95af714 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -384,4 +384,29 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]] top(num, comp) } + + /** + * Returns the top K elements from this RDD as defined by + * the specified Comparator[T] and maintains the order. + * @param num the number of top elements to return + * @param comp the comparator that defines the order + * @return an array of top elements + */ + def takeOrdered(num: Int, comp: Comparator[T]): JList[T] = { + import scala.collection.JavaConversions._ + val topElems = rdd.takeOrdered(num)(Ordering.comparatorToOrdering(comp)) + val arr: java.util.Collection[T] = topElems.toSeq + new java.util.ArrayList(arr) + } + + /** + * Returns the top K elements from this RDD using the + * natural ordering for T while maintain the order. + * @param num the number of top elements to return + * @return an array of top elements + */ + def takeOrdered(num: Int): JList[T] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]] + takeOrdered(num, comp) + } } From cfb6447ac4903a870dd268836dc0d8952491d591 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 11:47:57 -0700 Subject: [PATCH 162/419] Fixed for nonexistent bytes, added unit tests, changed stdout-page to stdout --- core/src/main/scala/spark/Utils.scala | 6 ++-- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 18 ++++++++---- core/src/test/scala/spark/ui/UISuite.scala | 28 +++++++++++++++++++ 5 files changed, 46 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 849edc13ee..512ac92d89 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -625,8 +625,10 @@ private object Utils extends Logging { def offsetBytes(path: String, a: Long, b: Long): String = { val file = new File(path) val length = file.length() - val buff = new Array[Byte](math.min((b-a).toInt, length.toInt)) - val skip = math.max(0, a) + val B = math.min(length, b) + val A = math.max(0, a) + val buff = new Array[Byte]((B-A).toInt) + val skip = A val stream = new FileInputStream(file) stream.skip(skip) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 49ced0d320..dae9995779 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -38,7 +38,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { state.completedApps.find(_.id == appId).getOrElse(null) }) - val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") val executors = app.executors.values.toSeq val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index af9943853f..a9c6c6d519 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -29,7 +29,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") val runningExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) val finishedExecutorTable = diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 602881d5f1..24356b0f63 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -71,7 +71,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0).asInstanceOf[Long] + val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0L) val maxBytes = 1024 * 1024 val defaultBytes = 100 * 1024 @@ -80,12 +80,18 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val logLength = new File(path).length() val logPageLength = math.min(byteLength, maxBytes) - val logText = {Utils.offsetBytes(path, offset, offset+logPageLength)} + + val fixedOffset = + if (offset < 0) 0 + else if (offset > logLength) logLength + else offset + + val logText = {Utils.offsetBytes(path, fixedOffset, fixedOffset+logPageLength)} val backButton = - if (offset > 0) { + if (fixedOffset > 0) { + .format(appId, executorId, logType, math.max(fixedOffset-logPageLength, 0), logPageLength)}> } @@ -94,9 +100,9 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } val nextButton = - if (offset+logPageLength < logLength) { + if (fixedOffset+logPageLength < logLength) { + format(appId, executorId, logType, fixedOffset+logPageLength, logPageLength)}> } diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index e4bb3abc33..b7a822c4bc 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -74,4 +74,32 @@ class UISuite extends FunSuite { FileUtils.deleteDirectory(tmpDir) } + + test("reading offset bytes of a file") { + val tmpDir2 = Files.createTempDir() + val f1Path = tmpDir2 + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) + f1.close() + + // Read first few bytes + assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") + + // Read some middle bytes + assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") + + // Read last few bytes + assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") + + //Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") + + //Read some nonexistent bytes at the end + assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") + + //Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + + FileUtils.deleteDirectory(tmpDir2) + } } From 04263e4d4609355d8e0779dea472fbb1aff5ef4f Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:15:42 -0700 Subject: [PATCH 163/419] Made some minor style changes --- core/src/main/scala/spark/Utils.scala | 11 +++++------ core/src/test/scala/spark/ui/UISuite.scala | 6 +++--- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 512ac92d89..c5627c8419 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -622,16 +622,15 @@ private object Utils extends Logging { } /** Return a string containing part of a file from byte 'a' to 'b'. */ - def offsetBytes(path: String, a: Long, b: Long): String = { + def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) val length = file.length() - val B = math.min(length, b) - val A = math.max(0, a) - val buff = new Array[Byte]((B-A).toInt) - val skip = A + val effectiveStart = math.min(length, start) + val effectiveEnd = math.max(0, end) + val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) val stream = new FileInputStream(file) - stream.skip(skip) + stream.skip(effectiveStart) stream.read(buff) stream.close() Source.fromBytes(buff).mkString diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index b7a822c4bc..ab174732e3 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -91,13 +91,13 @@ class UISuite extends FunSuite { // Read last few bytes assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") - //Read some nonexistent bytes in the beginning + // Read some nonexistent bytes in the beginning assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") - //Read some nonexistent bytes at the end + // Read some nonexistent bytes at the end assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") - //Read some nonexistent bytes on both ends + // Read some nonexistent bytes on both ends assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") FileUtils.deleteDirectory(tmpDir2) From 0d4580360ba4bbcd2f73877743d746c071a92e34 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:24:26 -0700 Subject: [PATCH 164/419] Fixed docstring of offsetBytes to match params and wrapped for 100+ character lines --- core/src/main/scala/spark/Utils.scala | 2 +- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c5627c8419..1c5af2700a 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,7 +621,7 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } - /** Return a string containing part of a file from byte 'a' to 'b'. */ + /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) val length = file.length() diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 24356b0f63..1f4ba5de60 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -91,7 +91,8 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val backButton = if (fixedOffset > 0) { + .format(appId, executorId, logType, math.max(fixedOffset-logPageLength, 0), + logPageLength)}> } From 5f8a20b4a8a4cb7e4b14ee989f1b4f83981ce61a Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:53:39 -0700 Subject: [PATCH 165/419] Moved unit tests for Utils from UISuite to UtilsSuite --- core/src/test/scala/spark/UtilsSuite.scala | 71 ++++++++++++++++++++++ core/src/test/scala/spark/ui/UISuite.scala | 71 ---------------------- 2 files changed, 71 insertions(+), 71 deletions(-) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 4a113e16bf..73050a1d3b 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -71,5 +71,76 @@ class UtilsSuite extends FunSuite { assert(Utils.splitCommandString("''") === Seq("")) assert(Utils.splitCommandString("\"\"") === Seq("")) } + + test("string formatting of time durations") { + val second = 1000 + val minute = second * 60 + val hour = minute * 60 + def str = Utils.msDurationToString(_) + + assert(str(123) === "123 ms") + assert(str(second) === "1.0 s") + assert(str(second + 462) === "1.5 s") + assert(str(hour) === "1.00 h") + assert(str(minute) === "1.0 m") + assert(str(minute + 4 * second + 34) === "1.1 m") + assert(str(10 * hour + minute + 4 * second) === "10.02 h") + assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") + } + + test("reading last n bytes of a file") { + val tmpDir = Files.createTempDir() + + // File smaller than limit + val f1Path = tmpDir + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) + f1.close() + assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") + + // File larger than limit + val f2Path = tmpDir + "/f2" + val f2 = new FileOutputStream(f2Path) + f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f2.close() + assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") + + // Request limit too + val f3Path = tmpDir + "/f2" + val f3 = new FileOutputStream(f3Path) + f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f3.close() + assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") + + FileUtils.deleteDirectory(tmpDir) + } + + test("reading offset bytes of a file") { + val tmpDir2 = Files.createTempDir() + val f1Path = tmpDir2 + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) + f1.close() + + // Read first few bytes + assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") + + // Read some middle bytes + assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") + + // Read last few bytes + assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") + + // Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") + + // Read some nonexistent bytes at the end + assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") + + // Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + + FileUtils.deleteDirectory(tmpDir2) + } } diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index ab174732e3..ef2e9c2ec4 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -31,75 +31,4 @@ class UISuite extends FunSuite { case Failure (e) => } } - - test("string formatting of time durations") { - val second = 1000 - val minute = second * 60 - val hour = minute * 60 - def str = Utils.msDurationToString(_) - - assert(str(123) === "123 ms") - assert(str(second) === "1.0 s") - assert(str(second + 462) === "1.5 s") - assert(str(hour) === "1.00 h") - assert(str(minute) === "1.0 m") - assert(str(minute + 4 * second + 34) === "1.1 m") - assert(str(10 * hour + minute + 4 * second) === "10.02 h") - assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") - } - - test("reading last n bytes of a file") { - val tmpDir = Files.createTempDir() - - // File smaller than limit - val f1Path = tmpDir + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) - f1.close() - assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") - - // File larger than limit - val f2Path = tmpDir + "/f2" - val f2 = new FileOutputStream(f2Path) - f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f2.close() - assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") - - // Request limit too - val f3Path = tmpDir + "/f2" - val f3 = new FileOutputStream(f3Path) - f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f3.close() - assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") - - FileUtils.deleteDirectory(tmpDir) - } - - test("reading offset bytes of a file") { - val tmpDir2 = Files.createTempDir() - val f1Path = tmpDir2 + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) - f1.close() - - // Read first few bytes - assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") - - // Read some middle bytes - assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") - - // Read last few bytes - assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") - - // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") - - // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") - - // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") - - FileUtils.deleteDirectory(tmpDir2) - } } From dbe948d9a2198845feb7c8bfa738286db41344d9 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 14:15:41 -0700 Subject: [PATCH 166/419] Moved appropriate import files from UISuite to UtilsSuite --- core/src/test/scala/spark/UtilsSuite.scala | 4 ++++ core/src/test/scala/spark/ui/UISuite.scala | 5 ----- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 73050a1d3b..942702c826 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -3,6 +3,10 @@ package spark import org.scalatest.FunSuite import java.io.{ByteArrayOutputStream, ByteArrayInputStream} import scala.util.Random +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import java.io.{FileOutputStream, File} +import com.google.common.base.Charsets class UtilsSuite extends FunSuite { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index ef2e9c2ec4..cd7d105928 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -4,11 +4,6 @@ import org.scalatest.FunSuite import org.eclipse.jetty.server.Server import java.net.ServerSocket import scala.util.{Failure, Success, Try} -import spark.Utils -import com.google.common.io.Files -import org.apache.commons.io.FileUtils -import java.io.{FileOutputStream, File} -import com.google.common.base.Charsets class UISuite extends FunSuite { test("jetty port increases under contention") { From f5f3b272f874a3ac22f4e0b2caab9053657a9124 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 14:52:29 -0700 Subject: [PATCH 167/419] Fixed mixup of start/end, moved more import files --- core/src/main/scala/spark/Utils.scala | 4 ++-- core/src/test/scala/spark/UtilsSuite.scala | 11 +++++------ core/src/test/scala/spark/ui/UISuite.scala | 4 ++-- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1c5af2700a..1da9c9574e 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -625,8 +625,8 @@ private object Utils extends Logging { def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) val length = file.length() - val effectiveStart = math.min(length, start) - val effectiveEnd = math.max(0, end) + val effectiveEnd = math.min(length, end) + val effectiveStart = math.max(0, start) val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) val stream = new FileInputStream(file) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 942702c826..d83a0307fa 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -1,12 +1,11 @@ package spark -import org.scalatest.FunSuite -import java.io.{ByteArrayOutputStream, ByteArrayInputStream} -import scala.util.Random -import com.google.common.io.Files -import org.apache.commons.io.FileUtils -import java.io.{FileOutputStream, File} import com.google.common.base.Charsets +import com.google.common.io.Files +import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream, File} +import org.scalatest.FunSuite +import org.apache.commons.io.FileUtils +import scala.util.Random class UtilsSuite extends FunSuite { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index cd7d105928..fc0c160720 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -1,9 +1,9 @@ package spark.ui +import scala.util.{Failure, Success, Try} +import java.net.ServerSocket import org.scalatest.FunSuite import org.eclipse.jetty.server.Server -import java.net.ServerSocket -import scala.util.{Failure, Success, Try} class UISuite extends FunSuite { test("jetty port increases under contention") { From 24196c91f0bd560c08399eed3cb1903248ca497a Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 15:27:52 -0700 Subject: [PATCH 168/419] Changed buffer to 10,000 bytes, created scrollbar for fixed-height log --- .../main/scala/spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- .../src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index dae9995779..27549e3b4a 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.memory} {executor.state} - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index a9c6c6d519..313ecec084 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -88,9 +88,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 1f4ba5de60..490fe15fcf 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -118,9 +118,9 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option {backButton} {nextButton}

    -
    {logText}
    - {backButton} - {nextButton} +
    +
    {logText}
    +
    UIUtils.basicSparkPage(content, "Log Page for " + appId) From 74bd3fc6806a4185a95a19c107e1a4776582db44 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 15:44:28 -0700 Subject: [PATCH 169/419] Added byte range on log pages --- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 490fe15fcf..cac281eef6 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -86,7 +86,10 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option else if (offset > logLength) logLength else offset - val logText = {Utils.offsetBytes(path, fixedOffset, fixedOffset+logPageLength)} + val endOffset = math.min(fixedOffset+logPageLength, logLength) + + val range =

    Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength}

    + val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} val backButton = if (fixedOffset > 0) { @@ -101,9 +104,9 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } val nextButton = - if (fixedOffset+logPageLength < logLength) { + if (endOffset < logLength) { + format(appId, executorId, logType, endOffset, logPageLength)}> } @@ -114,6 +117,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val content = + {range}
    {backButton} {nextButton} From ebf5b8c7c5fe72fbe2d74a1bfa67cbdc9f9be0ea Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Wed, 10 Jul 2013 22:16:06 -0700 Subject: [PATCH 170/419] Updating README to reflect Scala 2.9.3 requirements --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index ba24ab43b1..1dd96a0a4a 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,7 @@ This README file only contains basic setup instructions. ## Building -Spark requires Scala 2.9.2 (Scala 2.10 is not yet supported). The project is +Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The project is built using Simple Build Tool (SBT), which is packaged with it. To build Spark and its example programs, run: From 0ecc33f0c80733ff88518157be543ec458a76bdb Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 11:25:58 -0700 Subject: [PATCH 171/419] Added byte range, page title with log name, previous/next bytes buttons, initialization to end of log, large default buffer, buggy back to master link --- .../deploy/master/ui/ApplicationPage.scala | 4 +- .../scala/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/ui/IndexPage.scala | 9 +++-- .../spark/deploy/worker/ui/WorkerWebUI.scala | 37 +++++++++++-------- 4 files changed, 30 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 27549e3b4a..b2589abb89 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.memory} {executor.state} - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 6ae1cef940..f20ea42d7f 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -77,7 +77,7 @@ private[spark] class Worker( sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() - webUi = new WorkerWebUI(self, workDir, Some(webUiPort)) + webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 313ecec084..fc1ec31b4d 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -16,17 +16,18 @@ import spark.Utils import spark.ui.UIUtils private[spark] class IndexPage(parent: WorkerWebUI) { + val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout def renderJson(request: HttpServletRequest): JValue = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") @@ -88,9 +89,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index cac281eef6..3ab0f2eded 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -9,6 +9,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} +import spark.deploy.worker.Worker import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ @@ -18,7 +19,7 @@ import spark.ui.UIUtils * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option[Int] = None) +class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) extends Logging { implicit val timeout = Timeout( Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) @@ -71,14 +72,14 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0L) val maxBytes = 1024 * 1024 - val defaultBytes = 100 * 1024 + val defaultBytes = 10000 val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val logLength = new File(path).length() + val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(logLength-10000) val logPageLength = math.min(byteLength, maxBytes) val fixedOffset = @@ -88,46 +89,52 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val endOffset = math.min(fixedOffset+logPageLength, logLength) - val range =

    Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength}

    - val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} + val linkToMaster =

    Back to Master

    + + val range = Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength} val backButton = if (fixedOffset > 0) { - + } else { - + } val nextButton = if (endOffset < logLength) { - + } else { - + } + val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} + val content = - {range} -
    - {backButton} - {nextButton} -

    + {linkToMaster} +
    +
    +
    {backButton}
    +
    {range}
    +
    {nextButton}
    +
    +
    {logText}
    - UIUtils.basicSparkPage(content, "Log Page for " + appId) + UIUtils.basicSparkPage(content, logType + " log Page for " + appId) } def stop() { From 044d4577ec564fe0f0bd9c1ad86f4749b50fc100 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 12:02:15 -0700 Subject: [PATCH 172/419] Fixed capitalization of log page --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 3ab0f2eded..923b835c3f 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -134,7 +134,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - UIUtils.basicSparkPage(content, logType + " log Page for " + appId) + UIUtils.basicSparkPage(content, logType + " log page for " + appId) } def stop() { From e3a3fcf61b117d63db33ee28928dfd77cfd935b8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 12:16:38 -0700 Subject: [PATCH 173/419] Scrollbar on log pages appear automatically --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 923b835c3f..a3cbe4f5d3 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -129,7 +129,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
    {nextButton}

    -
    +
    {logText}
    From 11872888ca44d0c08c157973a8e35d344b0119e4 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 14:56:37 -0700 Subject: [PATCH 174/419] Created getByteRange function for logs and log pages, removed lastNBytes function --- core/src/main/scala/spark/Utils.scala | 30 +++++++++--- .../spark/deploy/worker/ui/WorkerWebUI.scala | 47 +++++++++---------- 2 files changed, 45 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1da9c9574e..04041d1179 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,6 +621,29 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } + /** Determine the byte range for a log or log page. */ + def getByteRange(path: String, offset: Option[String], byteLength: Option[String]) + : (Long, Long, Long, Int) = { + val defaultBytes = 10000 + val maxBytes = 1024 * 1024 + + val file = new File(path) + val logLength = file.length() + val getOffset = offset.map(_.toLong).getOrElse(logLength-defaultBytes) + + val fixedOffset = + if (getOffset < 0) 0L + else if (getOffset > logLength) logLength + else getOffset + + val getByteLength = byteLength.map(_.toInt).getOrElse(defaultBytes) + val logPageLength = math.min(getByteLength, maxBytes) + + val endOffset = math.min(fixedOffset+logPageLength, logLength) + + (fixedOffset, endOffset, logLength, logPageLength) + } + /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) @@ -636,13 +659,6 @@ private object Utils extends Logging { Source.fromBytes(buff).mkString } - /** Return a string containing the last `n` bytes of a file. */ - def lastNBytes(path: String, n: Int): String = { - val file = new File(path) - val length = file.length() - offsetBytes(path, length-n, length) - } - /** * Clone an object using a Spark serializer. */ diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index a3cbe4f5d3..f8ac682dbf 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -57,41 +57,39 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - - val maxBytes = 1024 * 1024 // Guard against OOM - val defaultBytes = 100 * 1024 - val numBytes = Option(request.getParameter("numBytes")) - .flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) - + val offset = Option(request.getParameter("offset")) + val byteLength = Option(request.getParameter("byteLength")) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val pre = "==== Last %s bytes of %s/%s/%s ====\n".format(numBytes, appId, executorId, logType) - pre + Utils.lastNBytes(path, math.min(numBytes, maxBytes)) + + val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val fixedOffset = offsetBytes._1 + val endOffset = offsetBytes._2 + val logLength = offsetBytes._3 + + val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n" + .format(fixedOffset, endOffset, logLength, appId, executorId, logType) + pre + Utils.offsetBytes(path, fixedOffset, endOffset) } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - - val maxBytes = 1024 * 1024 - val defaultBytes = 10000 - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - + val offset = Option(request.getParameter("offset")) + val byteLength = Option(request.getParameter("byteLength")) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val logLength = new File(path).length() - val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(logLength-10000) - val logPageLength = math.min(byteLength, maxBytes) - val fixedOffset = - if (offset < 0) 0 - else if (offset > logLength) logLength - else offset + val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val fixedOffset = offsetBytes._1 + val endOffset = offsetBytes._2 + val logLength = offsetBytes._3 + val logPageLength = offsetBytes._4 - val endOffset = math.min(fixedOffset+logPageLength, logLength) + val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} val linkToMaster =

    Back to Master

    - val range = Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength} + val range = Bytes {fixedOffset.toString} - {endOffset.toString} of {logLength} val backButton = if (fixedOffset > 0) { @@ -116,8 +114,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } - val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} - val content = @@ -134,7 +130,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
    - UIUtils.basicSparkPage(content, logType + " log page for " + appId) + UIUtils.basicSparkPage(content, request.getParameter("logType") + " log page for " + + request.getParameter("appId")) } def stop() { From 15fd11d65735265f6fcc25419b43c8c104a4bd17 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:01:50 -0700 Subject: [PATCH 175/419] Removed redundant calls to request by logPage --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index f8ac682dbf..676de231f0 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -130,8 +130,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - UIUtils.basicSparkPage(content, request.getParameter("logType") + " log page for " + - request.getParameter("appId")) + UIUtils.basicSparkPage(content, logType + " log page for " + appId) } def stop() { From 5d5dbc39f603201a3596031542d52742cfb30139 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:22:45 -0700 Subject: [PATCH 176/419] getByteRange moved to WorkerWebUI, takes converted parameters, returns only start/end offset --- core/src/main/scala/spark/Utils.scala | 23 ---------- .../spark/deploy/worker/ui/WorkerWebUI.scala | 43 +++++++++++++++---- 2 files changed, 34 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 04041d1179..5a37532306 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,29 +621,6 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } - /** Determine the byte range for a log or log page. */ - def getByteRange(path: String, offset: Option[String], byteLength: Option[String]) - : (Long, Long, Long, Int) = { - val defaultBytes = 10000 - val maxBytes = 1024 * 1024 - - val file = new File(path) - val logLength = file.length() - val getOffset = offset.map(_.toLong).getOrElse(logLength-defaultBytes) - - val fixedOffset = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset - - val getByteLength = byteLength.map(_.toInt).getOrElse(defaultBytes) - val logPageLength = math.min(getByteLength, maxBytes) - - val endOffset = math.min(fixedOffset+logPageLength, logLength) - - (fixedOffset, endOffset, logLength, logPageLength) - } - /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 676de231f0..1fb59de1d8 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -57,14 +57,15 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")) - val byteLength = Option(request.getParameter("byteLength")) + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val offsetBytes = getByteRange(path, offset, byteLength) val fixedOffset = offsetBytes._1 val endOffset = offsetBytes._2 - val logLength = offsetBytes._3 + val file = new File(path) + val logLength = file.length val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n" .format(fixedOffset, endOffset, logLength, appId, executorId, logType) @@ -75,15 +76,16 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")) - val byteLength = Option(request.getParameter("byteLength")) + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val offsetBytes = getByteRange(path, offset, byteLength) val fixedOffset = offsetBytes._1 val endOffset = offsetBytes._2 - val logLength = offsetBytes._3 - val logPageLength = offsetBytes._4 + val file = new File(path) + val logLength = file.length + val logPageLength = endOffset-fixedOffset val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} @@ -133,6 +135,29 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I UIUtils.basicSparkPage(content, logType + " log page for " + appId) } + /** Determine the byte range for a log or log page. */ + def getByteRange(path: String, offset: Option[Long], byteLength: Option[Int]) + : (Long, Long) = { + val defaultBytes = 10000 + val maxBytes = 1024 * 1024 + + val file = new File(path) + val logLength = file.length() + val getOffset = offset.getOrElse(logLength-defaultBytes) + + val fixedOffset = + if (getOffset < 0) 0L + else if (getOffset > logLength) logLength + else getOffset + + val getByteLength = byteLength.getOrElse(defaultBytes) + val logPageLength = math.min(getByteLength, maxBytes) + + val endOffset = math.min(fixedOffset+logPageLength, logLength) + + (fixedOffset, endOffset) + } + def stop() { server.foreach(_.stop()) } From fdc226a14cfdb5f699627191f1682763b8571126 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:36:43 -0700 Subject: [PATCH 177/419] Clarified start and end byte variable names --- .../spark/deploy/worker/ui/WorkerWebUI.scala | 36 +++++++++---------- 1 file changed, 16 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 1fb59de1d8..2a82c35231 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -61,15 +61,13 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = getByteRange(path, offset, byteLength) - val fixedOffset = offsetBytes._1 - val endOffset = offsetBytes._2 + val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n" - .format(fixedOffset, endOffset, logLength, appId, executorId, logType) - pre + Utils.offsetBytes(path, fixedOffset, endOffset) + .format(startByte, endByte, logLength, appId, executorId, logType) + pre + Utils.offsetBytes(path, startByte, endByte) } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { @@ -80,25 +78,23 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = getByteRange(path, offset, byteLength) - val fixedOffset = offsetBytes._1 - val endOffset = offsetBytes._2 + val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - val logPageLength = endOffset-fixedOffset + val logPageLength = endByte-startByte - val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} + val logText = {Utils.offsetBytes(path, startByte, endByte)} val linkToMaster =

    Back to Master

    - val range = Bytes {fixedOffset.toString} - {endOffset.toString} of {logLength} + val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} val backButton = - if (fixedOffset > 0) { + if (startByte > 0) { - + } else { @@ -106,10 +102,10 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } val nextButton = - if (endOffset < logLength) { + if (endByte < logLength) { - + format(appId, executorId, logType, endByte, logPageLength)}> + } else { @@ -145,7 +141,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val logLength = file.length() val getOffset = offset.getOrElse(logLength-defaultBytes) - val fixedOffset = + val startByte = if (getOffset < 0) 0L else if (getOffset > logLength) logLength else getOffset @@ -153,9 +149,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val getByteLength = byteLength.getOrElse(defaultBytes) val logPageLength = math.min(getByteLength, maxBytes) - val endOffset = math.min(fixedOffset+logPageLength, logLength) + val endByte = math.min(startByte+logPageLength, logLength) - (fixedOffset, endOffset) + (startByte, endByte) } def stop() { From 9ed036ccdbb1c7aa3279998f3177c1be6b01e16b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:33:53 -0700 Subject: [PATCH 178/419] Replaced logPageLength with byteLength to prevent buffer shrink bug --- .../spark/deploy/worker/ui/WorkerWebUI.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 2a82c35231..94d919f57a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -54,11 +54,12 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def log(request: HttpServletRequest): String = { + val defaultBytes = 10000 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val (startByte, endByte) = getByteRange(path, offset, byteLength) @@ -71,17 +72,18 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { + val defaultBytes = 10000 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - val logPageLength = endByte-startByte + //val logPageLength = endByte-startByte val logText = {Utils.offsetBytes(path, startByte, endByte)} @@ -92,9 +94,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val backButton = if (startByte > 0) { - + .format(appId, executorId, logType, math.max(startByte-byteLength, 0), + byteLength)}> + } else { @@ -104,8 +106,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val nextButton = if (endByte < logLength) { - + format(appId, executorId, logType, endByte, byteLength)}> + } else { @@ -132,7 +134,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } /** Determine the byte range for a log or log page. */ - def getByteRange(path: String, offset: Option[Long], byteLength: Option[Int]) + def getByteRange(path: String, offset: Option[Long], byteLength: Int) : (Long, Long) = { val defaultBytes = 10000 val maxBytes = 1024 * 1024 @@ -146,8 +148,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I else if (getOffset > logLength) logLength else getOffset - val getByteLength = byteLength.getOrElse(defaultBytes) - val logPageLength = math.min(getByteLength, maxBytes) + val logPageLength = math.min(byteLength, maxBytes) val endByte = math.min(startByte+logPageLength, logLength) From ece2388585a468dd57c973f254ba20c24299a464 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:35:56 -0700 Subject: [PATCH 179/419] Removed logPageLength from logPage --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 94d919f57a..1ab65d41e8 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -83,7 +83,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - //val logPageLength = endByte-startByte val logText = {Utils.offsetBytes(path, startByte, endByte)} From a32784109d11e061cdca2eced07ac01be2061056 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:57:55 -0700 Subject: [PATCH 180/419] Fixed links for "Back to Master" --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index dbdc8e1057..4dd6c448a9 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -71,7 +71,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } else { addWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) context.watch(sender) // This doesn't work with remote actors but helps for testing - sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort) + sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort.get) schedule() } } From 6d054487bf4c9e9aa9033187f73e4e9d8366bc37 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 17:12:17 -0700 Subject: [PATCH 181/419] Replace default buffer value to 100 GB, changed buttons to use String notation, removed default buffer parameter in UI URLs --- .../spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- .../scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 12 +++++++----- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index b2589abb89..8553377d8f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.memory} {executor.state} - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index fc1ec31b4d..c515f2e238 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -89,9 +89,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 1ab65d41e8..4e7f86d77a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -54,7 +54,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def log(request: HttpServletRequest): String = { - val defaultBytes = 10000 + val defaultBytes = 100 * 1024 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") @@ -72,7 +72,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { - val defaultBytes = 10000 + val defaultBytes = 100 * 1024 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") @@ -95,7 +95,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - + } else { @@ -106,7 +107,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I if (endByte < logLength) { - + } else { @@ -135,7 +137,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I /** Determine the byte range for a log or log page. */ def getByteRange(path: String, offset: Option[Long], byteLength: Int) : (Long, Long) = { - val defaultBytes = 10000 + val defaultBytes = 100 * 1024 val maxBytes = 1024 * 1024 val file = new File(path) From 5c67ca027834582bcd6cdd55baee5fd74e743a71 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 17:31:59 -0700 Subject: [PATCH 182/419] Remove "Bytes" in lieu of String notation --- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 4e7f86d77a..ccd55c1ce4 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -95,24 +95,22 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - + } else { - + } val nextButton = if (endByte < logLength) { - + } else { - + } val content = From a2c915fba84a1deadac199994322e0f15d5750b4 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 11 Jul 2013 18:55:00 -0700 Subject: [PATCH 183/419] giving order to top and making tests more clear --- core/src/main/scala/spark/RDD.scala | 6 ++---- core/src/test/scala/spark/RDDSuite.scala | 8 ++++---- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index af52040fa6..4c1591ed5a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -778,7 +778,7 @@ abstract class RDD[T: ClassManifest]( }.reduce { (queue1, queue2) => queue1 ++= queue2 queue1 - }.toArray + }.toArray.sorted(ord.reverse) } /** @@ -789,9 +789,7 @@ abstract class RDD[T: ClassManifest]( * @param ord the implicit ordering for T * @return an array of top elements */ - def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { - top(num)(ord.reverse).sorted(ord) - } + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse) /** * Save this RDD as a text file, using string representations of elements. diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index fe17d1d5e7..aa3ee5f5ee 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -240,7 +240,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) val topK = ints.top(5) assert(topK.size === 5) - assert(topK.sorted === nums.sorted.takeRight(5)) + assert(topK === nums.reverse.take(5)) } test("top with custom ordering") { @@ -255,9 +255,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("takeOrdered with predefined ordering") { val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) val rdd = sc.makeRDD(nums, 2) - val sortedTopK = rdd.takeOrdered(5) - assert(sortedTopK.size === 5) - assert(sortedTopK === Array(1, 2, 3, 4, 5)) + val sortedLowerK = rdd.takeOrdered(5) + assert(sortedLowerK.size === 5) + assert(sortedLowerK === Array(1, 2, 3, 4, 5)) } test("takeOrdered with custom ordering") { From 2080e250060975a876a388eb785e7f2b3cf2c0cd Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 12 Jul 2013 14:25:18 +0800 Subject: [PATCH 184/419] Enhance job ui in spark ui system with adding pool information --- core/src/main/scala/spark/SparkContext.scala | 17 ++- .../scala/spark/scheduler/DAGScheduler.scala | 4 +- .../scala/spark/scheduler/JobLogger.scala | 2 +- .../scala/spark/scheduler/SparkListener.scala | 2 +- .../scala/spark/scheduler/TaskScheduler.scala | 7 + .../scheduler/cluster/ClusterScheduler.scala | 11 +- .../cluster/ClusterTaskSetManager.scala | 4 +- .../spark/scheduler/cluster/Schedulable.scala | 6 +- .../scheduler/cluster/SchedulingMode.scala | 4 +- .../scheduler/cluster/TaskSetManager.scala | 1 + .../scheduler/local/LocalScheduler.scala | 10 +- .../scheduler/local/LocalTaskSetManager.scala | 1 + .../main/scala/spark/ui/jobs/IndexPage.scala | 115 ++++---------- .../spark/ui/jobs/JobProgressListener.scala | 140 +++++++++++++++++ .../scala/spark/ui/jobs/JobProgressUI.scala | 112 +++----------- .../main/scala/spark/ui/jobs/PoolPage.scala | 38 +++++ .../main/scala/spark/ui/jobs/PoolTable.scala | 98 ++++++++++++ .../main/scala/spark/ui/jobs/StageTable.scala | 143 ++++++++++++++++++ .../spark/scheduler/DAGSchedulerSuite.scala | 6 + .../spark/scheduler/JobLoggerSuite.scala | 2 +- 20 files changed, 527 insertions(+), 196 deletions(-) create mode 100644 core/src/main/scala/spark/ui/jobs/JobProgressListener.scala create mode 100644 core/src/main/scala/spark/ui/jobs/PoolPage.scala create mode 100644 core/src/main/scala/spark/ui/jobs/PoolTable.scala create mode 100644 core/src/main/scala/spark/ui/jobs/StageTable.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 43e6af2351..b5225d5681 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -10,6 +10,7 @@ import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.generic.Growable import scala.collection.mutable.HashMap +import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.util.DynamicVariable import scala.collection.mutable.{ConcurrentMap, HashMap} @@ -43,13 +44,14 @@ import org.apache.mesos.MesosNativeLibrary import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} -import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} +import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -540,6 +542,17 @@ class SparkContext( env.blockManager.master.getStorageStatus } + def getPoolsInfo: ArrayBuffer[Schedulable] = { + taskScheduler.rootPool.schedulableQueue + } + + def getSchedulingMode: SchedulingMode = { + taskScheduler.schedulingMode + } + + def getPoolNameToPool: HashMap[String, Schedulable] = { + taskScheduler.rootPool.schedulableNameToSchedulable + } /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 3d3b9ea011..c865743e37 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -472,11 +472,11 @@ class DAGScheduler( } } if (tasks.size > 0) { - sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) + val properties = idToActiveJob(stage.priority).properties + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size, properties))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) - val properties = idToActiveJob(stage.priority).properties taskSched.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority, properties)) if (!stage.submissionTime.isDefined) { diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 6a9d52f356..8e5540873f 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -45,7 +45,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { event match { case SparkListenerJobStart(job, properties) => processJobStartEvent(job, properties) - case SparkListenerStageSubmitted(stage, taskSize) => + case SparkListenerStageSubmitted(stage, taskSize, properties) => processStageSubmittedEvent(stage, taskSize) case StageCompleted(stageInfo) => processStageCompletedEvent(stageInfo) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 8de3aa91a4..94fdad9b98 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -8,7 +8,7 @@ import spark.executor.TaskMetrics sealed trait SparkListenerEvents -case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends SparkListenerEvents +case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties = null) extends SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala index 7787b54762..5cdf846032 100644 --- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala @@ -1,5 +1,7 @@ package spark.scheduler +import spark.scheduler.cluster.Pool +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Low-level task scheduler interface, implemented by both ClusterScheduler and LocalScheduler. * These schedulers get sets of tasks submitted to them from the DAGScheduler for each stage, @@ -8,6 +10,11 @@ package spark.scheduler * the TaskSchedulerListener interface. */ private[spark] trait TaskScheduler { + + def rootPool: Pool + + def schedulingMode: SchedulingMode + def start(): Unit // Invoked after system has successfully initialized (typically in spark context). diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 3a0c29b27f..1b23fd6cef 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -12,6 +12,7 @@ import spark.scheduler._ import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong import java.util.{TimerTask, Timer} +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call @@ -97,6 +98,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null + //default scheduler is FIFO + val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) override def setListener(listener: TaskSchedulerListener) { this.listener = listener @@ -104,15 +107,13 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def initialize(context: SchedulerBackend) { backend = context - //default scheduler is FIFO - val schedulingMode = System.getProperty("spark.cluster.schedulingmode", "FIFO") //temporarily set rootPool name to empty - rootPool = new Pool("", SchedulingMode.withName(schedulingMode), 0, 0) + rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { - case "FIFO" => + case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) - case "FAIR" => + case SchedulingMode.FAIR => new FairSchedulableBuilder(rootPool) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index fe6420a522..7a6a6b7826 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -90,8 +90,8 @@ private[spark] class ClusterTaskSetManager( var priority = taskSet.priority var stageId = taskSet.stageId var name = "TaskSet_"+taskSet.stageId.toString - var parent:Schedulable = null - + var parent: Schedulable = null + var schedulableQueue :ArrayBuffer[Schedulable] = null // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 2dd9c0564f..2e4f14c11f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,13 +1,17 @@ package spark.scheduler.cluster -import scala.collection.mutable.ArrayBuffer +import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import scala.collection.mutable.ArrayBuffer /** * An interface for schedulable entities. * there are two type of Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { var parent: Schedulable + //childrens + def schedulableQueue: ArrayBuffer[Schedulable] + def schedulingMode: SchedulingMode def weight: Int def minShare: Int def runningTasks: Int diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 6e0c6793e0..c5c7ee3b22 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,7 +1,7 @@ package spark.scheduler.cluster -object SchedulingMode extends Enumeration("FAIR","FIFO"){ +object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE"){ type SchedulingMode = Value - val FAIR,FIFO = Value + val FAIR,FIFO,NONE = Value } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index b4dd75d90f..472e01b227 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -6,6 +6,7 @@ import spark.TaskState.TaskState import java.nio.ByteBuffer private[spark] trait TaskSetManager extends Schedulable { + def schedulingMode = SchedulingMode.NONE def taskSet: TaskSet def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index b000e328e6..19a48895e3 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -12,6 +12,7 @@ import spark.TaskState.TaskState import spark.executor.ExecutorURLClassLoader import spark.scheduler._ import spark.scheduler.cluster._ +import spark.scheduler.cluster.SchedulingMode.SchedulingMode import akka.actor._ /** @@ -63,6 +64,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null + val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) val activeTaskSets = new HashMap[String, TaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -70,15 +72,13 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var localActor: ActorRef = null override def start() { - //default scheduler is FIFO - val schedulingMode = System.getProperty("spark.cluster.schedulingmode", "FIFO") //temporarily set rootPool name to empty - rootPool = new Pool("", SchedulingMode.withName(schedulingMode), 0, 0) + rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { - case "FIFO" => + case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) - case "FAIR" => + case SchedulingMode.FAIR => new FairSchedulableBuilder(rootPool) } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index f12fec41d5..8954f40ea9 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -14,6 +14,7 @@ import spark.scheduler.cluster._ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { var parent: Schedulable = null + var schedulableQueue :ArrayBuffer[Schedulable] = null var weight: Int = 1 var minShare: Int = 0 var runningTasks: Int = 0 diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 1e675ab2cb..e765cecb01 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -6,107 +6,52 @@ import javax.servlet.http.HttpServletRequest import scala.Some import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ import spark.storage.StorageLevel +import spark.scheduler.cluster.Schedulable +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode -/** Page showing list of all ongoing and recently finished stages */ +/** Page showing list of all ongoing and recently finished stages and pools*/ private[spark] class IndexPage(parent: JobProgressUI) { def listener = parent.listener - val dateFmt = parent.dateFmt + + def stageTable: StageTable = parent.stageTable + + def poolTable: PoolTable = parent.poolTable def render(request: HttpServletRequest): Seq[Node] = { val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - /** Special table which merges two header cells. */ - def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - - - - - - - - - - - {rows.map(r => makeRow(r))} - -
    Stage IdOriginSubmittedDurationTasks: Complete/TotalShuffle ActivityStored RDD
    - } + stageTable.setStagePoolInfo(parent.stagePoolInfo) + poolTable.setPoolSource(parent.stagePagePoolSource) - val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) - val completedStageTable = stageTable(stageRow, completedStages) - val failedStageTable: NodeSeq = stageTable(stageRow, failedStages) + val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) + val completedStageNodeSeq = stageTable.toNodeSeq(completedStages) + val failedStageNodeSeq = stageTable.toNodeSeq(failedStages) - val content =

    Active Stages

    ++ activeStageTable ++ -

    Completed Stages

    ++ completedStageTable ++ -

    Failed Stages

    ++ failedStageTable + val content =
    +
    +
      +
    • Active Stages Number: {activeStages.size}
    • +
    • Completed Stages Number: {completedStages.size}
    • +
    • Failed Stages Number: {failedStages.size}
    • +
    • Scheduling Mode: {parent.sc.getSchedulingMode}
    • +
    +
    +
    ++ +

    Pools

    ++ poolTable.toNodeSeq ++ +

    Active Stages : {activeStages.size}

    ++ activeStageNodeSeq++ +

    Completed Stages : {completedStages.size}

    ++ completedStageNodeSeq++ +

    Failed Stages : {failedStages.size}

    ++ failedStageNodeSeq - headerSparkPage(content, parent.sc, "Spark Stages", Jobs) - } - - def getElapsedTime(submitted: Option[Long], completed: Long): String = { - submitted match { - case Some(t) => parent.formatDuration(completed - t) - case _ => "Unknown" - } - } - - def makeProgressBar(completed: Int, total: Int): Seq[Node] = { - val width=130 - val height=15 - val completeWidth = (completed.toDouble / total) * width - - - - - - } - - - def stageRow(s: Stage): Seq[Node] = { - val submissionTime = s.submissionTime match { - case Some(t) => dateFmt.format(new Date(t)) - case None => "Unknown" - } - val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) - val shuffleInfo = (read, write) match { - case (true, true) => "Read/Write" - case (true, false) => "Read" - case (false, true) => "Write" - case _ => "" - } - val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) - val totalTasks = s.numPartitions - - - {s.id} - {s.origin} - {submissionTime} - {getElapsedTime(s.submissionTime, - s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeProgressBar(completedTasks, totalTasks)} - {completedTasks} / {totalTasks} - {listener.stageToTasksFailed.getOrElse(s.id, 0) match { - case f if f > 0 => "(%s failed)".format(f) - case _ => - }} - - {shuffleInfo} - {if (s.rdd.getStorageLevel != StorageLevel.NONE) { - - {Option(s.rdd.name).getOrElse(s.rdd.id)} - - }} - - + headerSparkPage(content, parent.sc, "Spark Stages/Pools", Jobs) } } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala new file mode 100644 index 0000000000..1244f9538b --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala @@ -0,0 +1,140 @@ +package spark.ui.jobs + +import scala.Seq +import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} + +import spark.{ExceptionFailure, SparkContext, Success, Utils} +import spark.scheduler._ +import spark.scheduler.cluster.TaskInfo +import spark.executor.TaskMetrics +import collection.mutable + +private[spark] class FairJobProgressListener(val sparkContext: SparkContext) + extends JobProgressListener(sparkContext) { + + val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" + val DEFAULT_POOL_NAME = "default" + + override val stageToPool = HashMap[Stage, String]() + override val poolToActiveStages = HashMap[String, HashSet[Stage]]() + + override def onStageCompleted(stageCompleted: StageCompleted) = { + super.onStageCompleted(stageCompleted) + val stage = stageCompleted.stageInfo.stage + poolToActiveStages(stageToPool(stage)) -= stage + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { + super.onStageSubmitted(stageSubmitted) + val stage = stageSubmitted.stage + var poolName = DEFAULT_POOL_NAME + if (stageSubmitted.properties != null) { + poolName = stageSubmitted.properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) + } + stageToPool(stage) = poolName + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) + stages += stage + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + super.onJobEnd(jobEnd) + jobEnd match { + case end: SparkListenerJobEnd => + end.jobResult match { + case JobFailed(ex, Some(stage)) => + poolToActiveStages(stageToPool(stage)) -= stage + case _ => + } + case _ => + } + } +} + +private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { + // How many stages to remember + val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt + + def stageToPool: HashMap[Stage, String] = null + def poolToActiveStages: HashMap[String, HashSet[Stage]] =null + + val activeStages = HashSet[Stage]() + val completedStages = ListBuffer[Stage]() + val failedStages = ListBuffer[Stage]() + + val stageToTasksComplete = HashMap[Int, Int]() + val stageToTasksFailed = HashMap[Int, Int]() + val stageToTaskInfos = + HashMap[Int, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + + override def onJobStart(jobStart: SparkListenerJobStart) {} + + override def onStageCompleted(stageCompleted: StageCompleted) = { + val stage = stageCompleted.stageInfo.stage + activeStages -= stage + completedStages += stage + trimIfNecessary(completedStages) + } + + /** If stages is too large, remove and garbage collect old stages */ + def trimIfNecessary(stages: ListBuffer[Stage]) { + if (stages.size > RETAINED_STAGES) { + val toRemove = RETAINED_STAGES / 10 + stages.takeRight(toRemove).foreach( s => { + stageToTaskInfos.remove(s.id) + }) + stages.trimEnd(toRemove) + } + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = + activeStages += stageSubmitted.stage + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val sid = taskEnd.task.stageId + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = + taskEnd.reason match { + case e: ExceptionFailure => + stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 + (Some(e), e.metrics) + case _ => + stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 + (None, Some(taskEnd.taskMetrics)) + } + val taskList = stageToTaskInfos.getOrElse( + sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskEnd.taskInfo, metrics, failureInfo)) + stageToTaskInfos(sid) = taskList + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + jobEnd match { + case end: SparkListenerJobEnd => + end.jobResult match { + case JobFailed(ex, Some(stage)) => + activeStages -= stage + failedStages += stage + trimIfNecessary(failedStages) + case _ => + } + case _ => + } + } + + /** Is this stage's input from a shuffle read. */ + def hasShuffleRead(stageID: Int): Boolean = { + // This is written in a slightly complicated way to avoid having to scan all tasks + for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { + if (s._2 != null) return s._2.flatMap(m => m.shuffleReadMetrics).isDefined + } + return false // No tasks have finished for this stage + } + + /** Is this stage's output to a shuffle write. */ + def hasShuffleWrite(stageID: Int): Boolean = { + // This is written in a slightly complicated way to avoid having to scan all tasks + for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { + if (s._2 != null) return s._2.flatMap(m => m.shuffleWriteMetrics).isDefined + } + return false // No tasks have finished for this stage + } +} diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 84730cc091..e610252242 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -14,9 +14,9 @@ import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} import spark.ui.JettyUtils._ import spark.{ExceptionFailure, SparkContext, Success, Utils} import spark.scheduler._ -import spark.scheduler.cluster.TaskInfo -import spark.executor.TaskMetrics import collection.mutable +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { @@ -24,104 +24,38 @@ private[spark] class JobProgressUI(val sc: SparkContext) { def listener = _listener.get val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) + private val poolPage = new PoolPage(this) + + var stageTable: StageTable = null + var stagePoolInfo: StagePoolInfo = null + var poolTable: PoolTable = null + var stagePagePoolSource: PoolSource = null def start() { - _listener = Some(new JobProgressListener) + sc.getSchedulingMode match { + case SchedulingMode.FIFO => + _listener = Some(new JobProgressListener(sc)) + stagePoolInfo = new FIFOStagePoolInfo() + stagePagePoolSource = new FIFOSource() + case SchedulingMode.FAIR => + _listener = Some(new FairJobProgressListener(sc)) + stagePoolInfo = new FairStagePoolInfo(listener) + stagePagePoolSource = new FairSource(sc) + } + sc.addSparkListener(listener) + stageTable = new StageTable(dateFmt, formatDuration, listener) + poolTable = new PoolTable(listener) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) def getHandlers = Seq[(String, Handler)]( ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), + ("/stages/pool", (request: HttpServletRequest) => poolPage.render(request)), ("/stages", (request: HttpServletRequest) => indexPage.render(request)) ) } - -private[spark] class JobProgressListener extends SparkListener { - // How many stages to remember - val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt - - val activeStages = HashSet[Stage]() - val completedStages = ListBuffer[Stage]() - val failedStages = ListBuffer[Stage]() - - val stageToTasksComplete = HashMap[Int, Int]() - val stageToTasksFailed = HashMap[Int, Int]() - val stageToTaskInfos = - HashMap[Int, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() - - override def onJobStart(jobStart: SparkListenerJobStart) {} - - override def onStageCompleted(stageCompleted: StageCompleted) = { - val stage = stageCompleted.stageInfo.stage - activeStages -= stage - completedStages += stage - trimIfNecessary(completedStages) - } - - /** If stages is too large, remove and garbage collect old stages */ - def trimIfNecessary(stages: ListBuffer[Stage]) { - if (stages.size > RETAINED_STAGES) { - val toRemove = RETAINED_STAGES / 10 - stages.takeRight(toRemove).foreach( s => { - stageToTaskInfos.remove(s.id) - }) - stages.trimEnd(toRemove) - } - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = - activeStages += stageSubmitted.stage - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val sid = taskEnd.task.stageId - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = - taskEnd.reason match { - case e: ExceptionFailure => - stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - (Some(e), e.metrics) - case _ => - stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - (None, Some(taskEnd.taskMetrics)) - } - val taskList = stageToTaskInfos.getOrElse( - sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) - taskList += ((taskEnd.taskInfo, metrics, failureInfo)) - stageToTaskInfos(sid) = taskList - } - - override def onJobEnd(jobEnd: SparkListenerJobEnd) { - jobEnd match { - case end: SparkListenerJobEnd => - end.jobResult match { - case JobFailed(ex, Some(stage)) => - activeStages -= stage - failedStages += stage - trimIfNecessary(failedStages) - case _ => - } - case _ => - } - } - - /** Is this stage's input from a shuffle read. */ - def hasShuffleRead(stageID: Int): Boolean = { - // This is written in a slightly complicated way to avoid having to scan all tasks - for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.flatMap(m => m.shuffleReadMetrics).isDefined - } - return false // No tasks have finished for this stage - } - - /** Is this stage's output to a shuffle write. */ - def hasShuffleWrite(stageID: Int): Boolean = { - // This is written in a slightly complicated way to avoid having to scan all tasks - for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.flatMap(m => m.shuffleWriteMetrics).isDefined - } - return false // No tasks have finished for this stage - } -} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala new file mode 100644 index 0000000000..00703887c3 --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -0,0 +1,38 @@ +package spark.ui.jobs + +import javax.servlet.http.HttpServletRequest + +import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashSet + +import spark.scheduler.Stage +import spark.ui.UIUtils._ +import spark.ui.Page._ + +/** Page showing specific pool details*/ +private[spark] class PoolPage(parent: JobProgressUI) { + def listener = parent.listener + + def stageTable: StageTable = parent.stageTable + + def poolTable: PoolTable = parent.poolTable + + def render(request: HttpServletRequest): Seq[Node] = { + val poolName = request.getParameter("poolname") + val poolToActiveStages = listener.poolToActiveStages + val activeStages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]).toSeq + val stageToPool = listener.stageToPool + + val poolDetailPoolSource = new PoolDetailSource(parent.sc, poolName) + poolTable.setPoolSource(poolDetailPoolSource) + + stageTable.setStagePoolInfo(parent.stagePoolInfo) + + val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) + + val content =

    Pool

    ++ poolTable.toNodeSeq ++ +

    Active Stages : {activeStages.size}

    ++ activeStageNodeSeq + + headerSparkPage(content, parent.sc, "Spark Pool Details", Jobs) + } +} diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala new file mode 100644 index 0000000000..bb8be4b26e --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -0,0 +1,98 @@ +package spark.ui.jobs + +import java.util.Date + +import javax.servlet.http.HttpServletRequest + +import scala.Some +import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet + +import spark.SparkContext +import spark.scheduler.Stage +import spark.ui.UIUtils._ +import spark.ui.Page._ +import spark.storage.StorageLevel +import spark.scheduler.cluster.Schedulable + +/* + * Interface for get pools seq showing on Index or pool detail page + */ + +private[spark] trait PoolSource { + def getPools: Seq[Schedulable] +} + +/* + * Pool source for FIFO scheduler algorithm on Index page + */ +private[spark] class FIFOSource() extends PoolSource{ + def getPools: Seq[Schedulable] = { + Seq[Schedulable]() + } +} + +/* + * Pool source for Fair scheduler algorithm on Index page + */ +private[spark] class FairSource(sc: SparkContext) extends PoolSource{ + def getPools: Seq[Schedulable] = { + sc.getPoolsInfo.toSeq + } +} + +/* + * specific pool info for pool detail page + */ +private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource{ + def getPools: Seq[Schedulable] = { + val pools = HashSet[Schedulable]() + pools += sc.getPoolNameToPool(poolName) + pools.toSeq + } +} + +/** Table showing list of pools */ +private[spark] class PoolTable(listener: JobProgressListener) { + + var poolSource: PoolSource = null + var poolToActiveStages: HashMap[String, HashSet[Stage]] = listener.poolToActiveStages + + def toNodeSeq: Seq[Node] = { + poolTable(poolRow, poolSource.getPools) + } + + def setPoolSource(poolSource: PoolSource) { + this.poolSource = poolSource + } + + //pool tables + def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[Stage]]) => Seq[Node], rows: Seq[Schedulable]): Seq[Node] = { + + + + + + + + + + + {rows.map(r => makeRow(r, poolToActiveStages))} + +
    Pool NameMinimum SharePool WeightActive StagesRunning TasksSchedulingMode
    + } + + def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[Stage]]): Seq[Node] = { + + {p.name} + {p.minShare} + {p.weight} + {poolToActiveStages.getOrElseUpdate(p.name, new HashSet[Stage]()).size} + {p.runningTasks} + {p.schedulingMode} + + } +} + diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala new file mode 100644 index 0000000000..83e566c55b --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -0,0 +1,143 @@ +package spark.ui.jobs + +import java.util.Date +import java.text.SimpleDateFormat + +import javax.servlet.http.HttpServletRequest + +import scala.Some +import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashMap + +import spark.scheduler.Stage +import spark.ui.UIUtils._ +import spark.ui.Page._ +import spark.storage.StorageLevel + +/* + * Interface to get stage's pool name + */ +private[spark] trait StagePoolInfo { + def getStagePoolName(s: Stage): String + + def hasHerf: Boolean +} + +/* + * For FIFO scheduler algorithm, just show "N/A" and its link status is false + */ +private[spark] class FIFOStagePoolInfo extends StagePoolInfo { + def getStagePoolName(s: Stage): String = "N/A" + + def hasHerf: Boolean = false +} + +/* + * For Fair scheduler algorithm, show its pool name and pool detail link status is true + */ +private[spark] class FairStagePoolInfo(listener: JobProgressListener) extends StagePoolInfo { + def getStagePoolName(s: Stage): String = { + listener.stageToPool(s) + } + + def hasHerf: Boolean = true +} + +/** Page showing list of all ongoing and recently finished stages */ +private[spark] class StageTable(val dateFmt: SimpleDateFormat, val formatDuration: Long => String, val listener: JobProgressListener) { + + var stagePoolInfo: StagePoolInfo = null + + def toNodeSeq(stages: Seq[Stage]): Seq[Node] = { + stageTable(stageRow, stages) + } + + def setStagePoolInfo(stagePoolInfo: StagePoolInfo) { + this.stagePoolInfo = stagePoolInfo + } + + /** Special table which merges two header cells. */ + def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + + + + + + + + + + + + {rows.map(r => makeRow(r))} + +
    Stage IdPool NameOriginSubmittedDurationTasks: Complete/TotalShuffle ActivityStored RDD
    + } + + def getElapsedTime(submitted: Option[Long], completed: Long): String = { + submitted match { + case Some(t) => formatDuration(completed - t) + case _ => "Unknown" + } + } + + def makeProgressBar(completed: Int, total: Int): Seq[Node] = { + val width=130 + val height=15 + val completeWidth = (completed.toDouble / total) * width + + + + + + } + + + def stageRow(s: Stage): Seq[Node] = { + val submissionTime = s.submissionTime match { + case Some(t) => dateFmt.format(new Date(t)) + case None => "Unknown" + } + val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) + val shuffleInfo = (read, write) match { + case (true, true) => "Read/Write" + case (true, false) => "Read" + case (false, true) => "Write" + case _ => "" + } + val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) + val totalTasks = s.numPartitions + + val poolName = stagePoolInfo.getStagePoolName(s) + + + {s.id} + {if (stagePoolInfo.hasHerf) { + {poolName} + } else { + {poolName} + }} + {s.origin} + {submissionTime} + {getElapsedTime(s.submissionTime, + s.completionTime.getOrElse(System.currentTimeMillis()))} + {makeProgressBar(completedTasks, totalTasks)} + {completedTasks} / {totalTasks} + {listener.stageToTasksFailed.getOrElse(s.id, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => + }} + + {shuffleInfo} + {if (s.rdd.getStorageLevel != StorageLevel.NONE) { + + {Option(s.rdd.name).getOrElse(s.rdd.id)} + + }} + + + } +} diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 30e6fef950..da72bfbf89 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -22,6 +22,10 @@ import spark.TaskEndReason import spark.{FetchFailed, Success} +import spark.scheduler.cluster.Pool +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode + /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler * rather than spawning an event loop thread as happens in the real code. They use EasyMock @@ -39,6 +43,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() val taskScheduler = new TaskScheduler() { + override def rootPool: Pool = null + override def schedulingMode: SchedulingMode = SchedulingMode.NONE override def start() = {} override def stop() = {} override def submitTasks(taskSet: TaskSet) = { diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 699901f1a1..328e7e7529 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -40,7 +40,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) - joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4)) + joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4, null)) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) parentRdd.setName("MyRDD") joblogger.getRddNameTest(parentRdd) should be ("MyRDD") From a1662326e9b1486361eba2f2caf903875fbba597 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 12 Jul 2013 08:38:19 -0700 Subject: [PATCH 185/419] comment adjustment to takeOrdered --- core/src/main/scala/spark/RDD.scala | 2 +- core/src/main/scala/spark/api/java/JavaRDDLike.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 4c1591ed5a..8aa77266bc 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -782,7 +782,7 @@ abstract class RDD[T: ClassManifest]( } /** - * Returns the top K elements from this RDD as defined by + * Returns the first K elements from this RDD as defined by * the specified implicit Ordering[T] and maintains the * ordering. * @param num the number of top elements to return diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 94b95af714..27f40ecdfd 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -386,7 +386,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the top K elements from this RDD as defined by + * Returns the first K elements from this RDD as defined by * the specified Comparator[T] and maintains the order. * @param num the number of top elements to return * @param comp the comparator that defines the order @@ -400,7 +400,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the top K elements from this RDD using the + * Returns the first K elements from this RDD using the * natural ordering for T while maintain the order. * @param num the number of top elements to return * @return an array of top elements From cd7259b4b8d8abbff6db963fd8f84d4bd0b3737b Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Fri, 12 Jul 2013 11:51:14 -0700 Subject: [PATCH 186/419] Fixes typos in Spark Streaming Programming Guide These typos were reported on the spark-users mailing list, see: https://groups.google.com/d/msg/spark-users/SyLGgJlKCrI/LpeBypOkSMUJ --- docs/streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index f5788dc467..8cd1b0cd66 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -7,7 +7,7 @@ title: Spark Streaming Programming Guide {:toc} # Overview -A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transformong existing DStreams using parallel operators like `map`, `reduce`, and `window`. The basic processing model is as follows: +A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transforming existing DStreams using parallel operators like `map`, `reduce`, and `window`. The basic processing model is as follows: (i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides it into batches. Each batch of data is treated as an RDD, that is, an immutable parallel collection of data. These input RDDs are saved in memory and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively called an InputDStream. (ii) Data received by InputDStreams are processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures. @@ -20,7 +20,7 @@ The first thing a Spark Streaming program must do is create a `StreamingContext` new StreamingContext(master, appName, batchDuration, [sparkHome], [jars]) {% endhighlight %} -The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). +The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such that the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). This constructor creates a SparkContext for your job as well, which can be accessed with `streamingContext.sparkContext`. From 531a7e55745ad76d17d41f80dbdea0072acce079 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 10 Jul 2013 17:59:43 -0700 Subject: [PATCH 187/419] Pass executor env vars (e.g. SPARK_CLASSPATH) to compute-classpath. --- core/src/main/scala/spark/Utils.scala | 12 +++++++++--- .../scala/spark/deploy/worker/ExecutorRunner.scala | 4 +++- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 64547bbdcd..d2bf151cbf 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -6,6 +6,7 @@ import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} import java.util.regex.Pattern +import scala.collection.Map import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConversions._ import scala.io.Source @@ -545,10 +546,15 @@ private object Utils extends Logging { /** * Execute a command and get its output, throwing an exception if it yields a code other than 0. */ - def executeAndGetOutput(command: Seq[String], workingDir: File = new File(".")): String = { - val process = new ProcessBuilder(command: _*) + def executeAndGetOutput(command: Seq[String], workingDir: File = new File("."), + extraEnvironment: Map[String, String] = Map.empty): String = { + val builder = new ProcessBuilder(command: _*) .directory(workingDir) - .start() + val environment = builder.environment() + for ((key, value) <- extraEnvironment) { + environment.put(key, value) + } + val process = builder.start() new Thread("read stderr for " + command(0)) { override def run() { for (line <- Source.fromInputStream(process.getErrorStream).getLines) { diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index d7f58b2cb1..5d3d54c65e 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -98,7 +98,9 @@ private[spark] class ExecutorRunner( // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" - val classPath = Utils.executeAndGetOutput(Seq(sparkHome + "/bin/compute-classpath" + ext)) + val classPath = Utils.executeAndGetOutput( + Seq(sparkHome + "/bin/compute-classpath" + ext), + extraEnvironment=appDesc.command.environment) Seq("-cp", classPath) ++ libraryOpts ++ userOpts ++ memoryOpts } From 73984b96a8450674b472676eaa855cc2df68a754 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 12 Jul 2013 14:26:56 -0700 Subject: [PATCH 188/419] Removed unit test of nonexistent function Utils.lastNBytes --- core/src/test/scala/spark/UtilsSuite.scala | 27 ---------------------- 1 file changed, 27 deletions(-) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index d83a0307fa..1e1260f606 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -91,33 +91,6 @@ class UtilsSuite extends FunSuite { assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") } - test("reading last n bytes of a file") { - val tmpDir = Files.createTempDir() - - // File smaller than limit - val f1Path = tmpDir + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) - f1.close() - assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") - - // File larger than limit - val f2Path = tmpDir + "/f2" - val f2 = new FileOutputStream(f2Path) - f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f2.close() - assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") - - // Request limit too - val f3Path = tmpDir + "/f2" - val f3 = new FileOutputStream(f3Path) - f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f3.close() - assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") - - FileUtils.deleteDirectory(tmpDir) - } - test("reading offset bytes of a file") { val tmpDir2 = Files.createTempDir() val f1Path = tmpDir2 + "/f1" From 6855338e1400638188358a7d7926eb86f668c160 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Jul 2013 19:24:16 -0700 Subject: [PATCH 189/419] Show block locations in Web UI. This fixes SPARK-769. Support is added for enumerating the locations of blocks in the UI. There is also some minor cleanup in StorageUtils. --- .../main/scala/spark/storage/StorageUtils.scala | 11 +++++++++-- .../main/scala/spark/ui/storage/RDDPage.scala | 17 +++++++++++++---- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 950c0cdf35..3e7fa287e5 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -39,12 +39,19 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, private[spark] object StorageUtils { - /* Given the current storage status of the BlockManager, returns information for each RDD */ - def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus], + /* Returns RDD-level information, compiled from a list of StorageStatus objects */ + def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus], sc: SparkContext) : Array[RDDInfo] = { rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) } + /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ + def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { + val blockLocationPairs = storageStatusList + .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) + blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap + } + /* Given a list of BlockStatus objets, returns information for each RDD */ def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], sc: SparkContext) : Array[RDDInfo] = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 0cb1e47ea5..428db6fa95 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -26,8 +26,14 @@ private[spark] class RDDPage(parent: BlockManagerUI) { val workers = filteredStorageStatusList.map((prefix, _)) val workerTable = listingTable(workerHeaders, workerRow, workers) - val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk") - val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk", + "Locations") + + val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) + val blocks = blockStatuses.map { + case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN"))) + } val blockTable = listingTable(blockHeaders, blockRow, blocks) val content = @@ -74,8 +80,8 @@ private[spark] class RDDPage(parent: BlockManagerUI) { headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name, Jobs) } - def blockRow(blk: (String, BlockStatus)): Seq[Node] = { - val (id, block) = blk + def blockRow(row: (String, BlockStatus, Seq[String])): Seq[Node] = { + val (id, block, locations) = row {id} @@ -87,6 +93,9 @@ private[spark] class RDDPage(parent: BlockManagerUI) { {Utils.memoryBytesToString(block.diskSize)} + + {locations.map(l => {l}
    )} + } From 08150f19abcf08f2a18305080b08736fb8a33a12 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Jul 2013 19:32:35 -0700 Subject: [PATCH 190/419] Minor style fix --- core/src/main/scala/spark/storage/StorageUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 3e7fa287e5..aca16f533a 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -47,8 +47,8 @@ object StorageUtils { /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { - val blockLocationPairs = storageStatusList - .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) + val blockLocationPairs = storageStatusList + .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap } From c4d5b01e44c5f289d67670185a14ff4ccd9537db Mon Sep 17 00:00:00 2001 From: seanm Date: Sat, 13 Jul 2013 14:56:23 -0600 Subject: [PATCH 191/419] changing com.google.code.findbugs maven coordinates --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 641d379749..0a4c176a29 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -144,7 +144,7 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", - "com.google.code.findbugs" % "jsr305" % "1.3.+", + "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.16", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, From 1889f9f450d3e28864a5ac7b5f9d3831cf7b17bf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 13 Jul 2013 14:43:21 -0700 Subject: [PATCH 192/419] Increase PermGen size --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 850c58e1e9..523fbb346b 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -5,4 +5,4 @@ if [ "$MESOS_HOME" != "" ]; then fi export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) export SPARK_TESTING=1 # To put test classes on classpath -java -Xmx1200m -XX:MaxPermSize=250m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" From ddb97f0fdf16442afaa9cab656376267e4044510 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 13 Jul 2013 16:05:21 -0700 Subject: [PATCH 193/419] Add `Environment` tab to SparkUI. This adds a tab which displays system property and classpath information. This can be useful in debugging various types of issues such as: 1. Extra/incorrect Hadoop jars being included in the classpath 2. Spark launching with a different JRE version than intended 3. Spark system properties not being set to intended values 4. User added jars that conflict with Spark jars --- core/src/main/scala/spark/ui/Page.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 7 +-- core/src/main/scala/spark/ui/UIUtils.scala | 5 ++ .../scala/spark/ui/env/EnvironmentUI.scala | 58 +++++++++++++++++++ 4 files changed, 67 insertions(+), 5 deletions(-) create mode 100644 core/src/main/scala/spark/ui/env/EnvironmentUI.scala diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala index c853b44b76..ed8f91842c 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/spark/ui/Page.scala @@ -1,3 +1,3 @@ package spark.ui -private[spark] object Page extends Enumeration { val Storage, Jobs = Value } \ No newline at end of file +private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index b3bdc2c490..874e5ba8ec 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -5,15 +5,13 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, SparkContext, Utils} +import spark.ui.env.EnvironmentUI import spark.ui.storage.BlockManagerUI import spark.ui.jobs.JobProgressUI -import spark.ui.UIUtils._ import spark.ui.JettyUtils._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { - // TODO(pwendell): It would be nice to add a view that prints out environment information - val host = Utils.localHostName() val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None @@ -25,7 +23,8 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) - val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers + val env = new EnvironmentUI(sc) + val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ handlers /** Bind the HTTP server which backs this web interface */ def bind() { diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index 7b79290d1b..36d9c47245 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -19,6 +19,10 @@ private[spark] object UIUtils { case Jobs =>
  • Jobs
  • case _ =>
  • Jobs
  • } + val environment = page match { + case Environment =>
  • Environment
  • + case _ =>
  • Environment
  • + } @@ -44,6 +48,7 @@ private[spark] object UIUtils {
    • Application: {sc.appName}
    • diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala new file mode 100644 index 0000000000..8a3b330405 --- /dev/null +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -0,0 +1,58 @@ +package spark.ui.env + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.collection.JavaConversions._ + +import spark.ui.JettyUtils._ +import spark.ui.UIUtils.headerSparkPage +import spark.ui.Page.Environment +import spark.SparkContext +import spark.ui.UIUtils + +import scala.xml.Node + +private[spark] class EnvironmentUI(sc: SparkContext) { + + def getHandlers = Seq[(String, Handler)]( + ("/environment", (request: HttpServletRequest) => envDetails(request)) + ) + + def envDetails(request: HttpServletRequest): Seq[Node] = { + val properties = System.getProperties.iterator.toSeq + + val classPathProperty = properties + .filter{case (k, v) => k.contains("java.class.path")} + .headOption + .getOrElse("", "") + val sparkProperties = properties.filter(_._1.contains("spark")) + val otherProperties = properties.diff(sparkProperties :+ classPathProperty) + + val propertyHeaders = Seq("Name", "Value") + def propertyRow(kv: (String, String)) = {kv._1}{kv._2} + val propertyTable = UIUtils.listingTable( + propertyHeaders, propertyRow, sparkProperties ++ otherProperties) + + val classPathEntries = classPathProperty._2 + .split(System.getProperty("path.separator", ":")) + .filterNot(e => e.isEmpty) + .map(e => (e, "System Classpath")) + val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} + val addedFiles = sc.addedFiles.iterator.toSeq.map{case (path, time) => (path, "Added By User")} + val classPath = addedJars ++ addedFiles ++ classPathEntries + + val classPathHeaders = Seq("Resource", "Source") + def classPathRow(data: (String, String)) = {data._1}{data._2} + val classPathTable = UIUtils.listingTable(classPathHeaders, classPathRow, classPath) + + val content = + +

      System Properties

      {propertyTable} +

      Classpath Entries

      {classPathTable} +
      + + headerSparkPage(content, sc, "Environment", Environment) + } +} From e271fde10b342216e33f4f45af73c5d103215cf2 Mon Sep 17 00:00:00 2001 From: root Date: Sun, 14 Jul 2013 06:24:29 +0000 Subject: [PATCH 194/419] Fixed a delay scheduling bug in the YARN branch, found by Patrick --- .../scheduler/cluster/ClusterTaskSetManager.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index fe6420a522..327d6797ae 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -454,10 +454,10 @@ private[spark] class ClusterTaskSetManager( val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val taskLocality = - if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else - if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else - if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else - TaskLocality.ANY + if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL + else if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL + else if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL + else TaskLocality.ANY val prefStr = taskLocality.toString logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( taskSet.id, index, taskId, execId, hostPort, prefStr)) @@ -467,7 +467,7 @@ private[spark] class ClusterTaskSetManager( val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) - if (TaskLocality.NODE_LOCAL == taskLocality) { + if (taskLocality == TaskLocality.PROCESS_LOCAL || taskLocality == TaskLocality.NODE_LOCAL) { lastPreferredLaunchTime = time } // Serialize and return the task From a44a7b123862202cf97f2de7a96aeaf29a93002a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:23:09 +0000 Subject: [PATCH 195/419] Determine Spark core classes better in getCallSite --- core/src/main/scala/spark/Utils.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c6a3f97872..a36186bf8a 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -579,8 +579,15 @@ private object Utils extends Logging { output.toString } + /** + * A regular expression to match classes of the "core" Spark API that we want to skip when + * finding the call site of a method. + */ + private val SPARK_CLASS_REGEX = """^spark(\.api\.java)?(\.rdd)?\.[A-Z]""".r + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, val firstUserLine: Int, val firstUserClass: String) + /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. @@ -602,7 +609,7 @@ private object Utils extends Logging { for (el <- trace) { if (!finished) { - if (el.getClassName.startsWith("spark.") && !el.getClassName.startsWith("spark.examples.")) { + if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName) != None) { lastSparkMethod = if (el.getMethodName == "") { // Spark method is a constructor; get its class name el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) From b91a218cea5a7ab4037675667922fc06bfec6fbf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:31:33 +0000 Subject: [PATCH 196/419] Cosmetic fixes to web UI --- .../src/main/scala/spark/deploy/master/ui/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index c6de2bafa3..7545ecf868 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -38,8 +38,8 @@ private[spark] class IndexPage(parent: MasterWebUI) {
        -
      • URL:{state.uri}
      • -
      • Workers:{state.workers.size}
      • +
      • URL: {state.uri}
      • +
      • Workers: {state.workers.size}
      • Cores: {state.workers.map(_.cores).sum} Total, {state.workers.map(_.coresUsed).sum} Used
      • Memory: diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 1e675ab2cb..7907ab3bc7 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -29,10 +29,10 @@ private[spark] class IndexPage(parent: JobProgressUI) { Stage Id Origin Submitted - Duration - Tasks: Complete/Total - Shuffle Activity - Stored RDD + Duration + Tasks: Complete/Total + Shuffle Activity + Stored RDD {rows.map(r => makeRow(r))} From c5c38d1987137a1dc5eb66dd1065735a542ef9b5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:59:50 +0000 Subject: [PATCH 197/419] Some optimizations to loading phase of ALS --- .../spark/mllib/recommendation/ALS.scala | 45 +++++++++++++------ 1 file changed, 32 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 6c9fb2359c..dec3701ec0 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -2,6 +2,7 @@ package spark.mllib.recommendation import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random +import scala.util.Sorting import spark.{HashPartitioner, Partitioner, SparkContext, RDD} import spark.storage.StorageLevel @@ -33,6 +34,12 @@ private[recommendation] case class InLinkBlock( elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[Double])]]) +/** + * A more compact class to represent a rating than Tuple3[Int, Int, Double]. + */ +private[recommendation] case class Rating(user: Int, product: Int, rating: Double) + + /** * Alternating Least Squares matrix factorization. * @@ -126,13 +133,13 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l * Make the out-links table for a block of the users (or products) dataset given the list of * (user, product, rating) values for the users in that block (or the opposite for products). */ - private def makeOutLinkBlock(numBlocks: Int, ratings: Array[(Int, Int, Double)]): OutLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted + private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating]): OutLinkBlock = { + val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) - for ((u, p, r) <- ratings) { - shouldSend(userIdToPos(u))(p % numBlocks) = true + for (r <- ratings) { + shouldSend(userIdToPos(r.user))(r.product % numBlocks) = true } OutLinkBlock(userIds, shouldSend) } @@ -141,18 +148,28 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l * Make the in-links table for a block of the users (or products) dataset given a list of * (user, product, rating) values for the users in that block (or the opposite for products). */ - private def makeInLinkBlock(numBlocks: Int, ratings: Array[(Int, Int, Double)]): InLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted + private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating]): InLinkBlock = { + val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap + // Split out our ratings by product block + val blockRatings = Array.fill(numBlocks)(new ArrayBuffer[Rating]) + for (r <- ratings) { + blockRatings(r.product % numBlocks) += r + } val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) for (productBlock <- 0 until numBlocks) { - val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) - val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) - .toArray - .sortBy(_._1) - .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} - ratingsForBlock(productBlock) = ratingsByProduct + // Create an array of (product, Seq(Rating)) ratings + val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray + // Sort them by user ID + val ordering = new Ordering[(Int, ArrayBuffer[Rating])] { + def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = a._1 - b._1 + } + Sorting.quickSort(groupedRatings)(ordering) + // Translate the user IDs to indices based on userIdToPos + ratingsForBlock(productBlock) = groupedRatings.map { case (p, rs) => + (rs.view.map(r => userIdToPos(r.user)).toArray, rs.view.map(_.rating).toArray) + } } InLinkBlock(userIds, ratingsForBlock) } @@ -167,7 +184,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l { val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { - val ratings = elements.map(_._2).toArray + val ratings = elements.map{case (k, t) => Rating(t._1, t._2, t._3)}.toArray val inLinkBlock = makeInLinkBlock(numBlocks, ratings) val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) @@ -373,6 +390,8 @@ object ALS { } val (master, ratingsFile, rank, iters, outputDir) = (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) + System.setProperty("spark.serializer", "spark.KryoSerializer") + System.setProperty("spark.locality.wait", "10000") val sc = new SparkContext(master, "ALS") val ratings = sc.textFile(ratingsFile).map { line => val fields = line.split(',') From 931e4c96ef56302cc384086d1faa0f4ca3258e8b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 08:03:13 +0000 Subject: [PATCH 198/419] Fix a comment --- mllib/src/main/scala/spark/mllib/recommendation/ALS.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index dec3701ec0..21eb21276e 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -161,7 +161,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l for (productBlock <- 0 until numBlocks) { // Create an array of (product, Seq(Rating)) ratings val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray - // Sort them by user ID + // Sort them by product ID val ordering = new Ordering[(Int, ArrayBuffer[Rating])] { def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = a._1 - b._1 } From 00556a94c98577d1536d0c2b4606a9933069a584 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Sun, 14 Jul 2013 17:04:53 +0800 Subject: [PATCH 199/419] add spaces before curly braces and after for if conditions --- .../spark/deploy/yarn/ApplicationMaster.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 68bb36d316..6a0617cc06 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -130,7 +130,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val t = new Thread { override def run() { var successed = false - try{ + try { // Copy var mainArgs: Array[String] = new Array[String](args.userArgs.size()) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) @@ -140,9 +140,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // It need shutdown hook to set SUCCEEDED successed = true } finally { - if(successed){ + if (successed) { ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - }else{ + } else { ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) } } @@ -190,7 +190,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("All workers have launched.") // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout - if (userThread.isAlive){ + if (userThread.isAlive) { // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) @@ -208,7 +208,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val t = new Thread { override def run() { - while (userThread.isAlive){ + while (userThread.isAlive) { val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning if (missingWorkerCount > 0) { logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") @@ -250,7 +250,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e def finishApplicationMaster(status: FinalApplicationStatus) { synchronized { - if(isFinished){ + if (isFinished) { return } isFinished = true @@ -276,7 +276,7 @@ object ApplicationMaster { private val ALLOCATOR_LOOP_WAIT_COUNT = 30 def incrementAllocatorLoop(by: Int) { val count = yarnAllocatorLoop.getAndAdd(by) - if (count >= ALLOCATOR_LOOP_WAIT_COUNT){ + if (count >= ALLOCATOR_LOOP_WAIT_COUNT) { yarnAllocatorLoop.synchronized { // to wake threads off wait ... yarnAllocatorLoop.notifyAll() @@ -320,7 +320,7 @@ object ApplicationMaster { // Wait for initialization to complete and atleast 'some' nodes can get allocated yarnAllocatorLoop.synchronized { - while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT){ + while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) { yarnAllocatorLoop.wait(1000L) } } From 4883586838b960c405af208262aed3ec4b078613 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 14 Jul 2013 10:37:26 -0700 Subject: [PATCH 200/419] Responding to Matei's review --- .../scala/spark/ui/env/EnvironmentUI.scala | 21 ++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index 8a3b330405..6b8b9f05bb 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -5,6 +5,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler import scala.collection.JavaConversions._ +import scala.util.Properties import spark.ui.JettyUtils._ import spark.ui.UIUtils.headerSparkPage @@ -21,19 +22,27 @@ private[spark] class EnvironmentUI(sc: SparkContext) { ) def envDetails(request: HttpServletRequest): Seq[Node] = { - val properties = System.getProperties.iterator.toSeq + val jvmInformation = Seq( + ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), + ("Java Home", Properties.javaHome), + ("Scala Version", Properties.versionString), + ("Scala Home", Properties.scalaHome) + ) + def jvmRow(kv: (String, String)) = {kv._1}{kv._2} + def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation) + val properties = System.getProperties.iterator.toSeq val classPathProperty = properties .filter{case (k, v) => k.contains("java.class.path")} .headOption .getOrElse("", "") - val sparkProperties = properties.filter(_._1.contains("spark")) + val sparkProperties = properties.filter(_._1.startsWith("spark")) val otherProperties = properties.diff(sparkProperties :+ classPathProperty) val propertyHeaders = Seq("Name", "Value") def propertyRow(kv: (String, String)) = {kv._1}{kv._2} - val propertyTable = UIUtils.listingTable( - propertyHeaders, propertyRow, sparkProperties ++ otherProperties) + val sparkPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties) + val otherPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties) val classPathEntries = classPathProperty._2 .split(System.getProperty("path.separator", ":")) @@ -49,7 +58,9 @@ private[spark] class EnvironmentUI(sc: SparkContext) { val content = -

        System Properties

        {propertyTable} +

        Runtime Information

        {jvmTable} +

        Spark Properties

        {sparkPropertyTable} +

        System Properties

        {otherPropertyTable}

        Classpath Entries

        {classPathTable}
        From ed7fd501cf7ece730cbdee6c152b917cf6bfb16a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 00:30:10 +0000 Subject: [PATCH 201/419] Make number of blocks in ALS configurable and lower the default --- .../src/main/scala/spark/mllib/recommendation/ALS.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 21eb21276e..2abaf2f2dd 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -91,7 +91,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l */ def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { val numBlocks = if (this.numBlocks == -1) { - math.max(ratings.context.defaultParallelism, ratings.partitions.size) + math.max(ratings.context.defaultParallelism, ratings.partitions.size / 2) } else { this.numBlocks } @@ -384,12 +384,13 @@ object ALS { } def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: ALS ") + if (args.length != 5 && args.length != 6) { + println("Usage: ALS []") System.exit(1) } val (master, ratingsFile, rank, iters, outputDir) = (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) + val blocks = if (args.length == 6) args(5).toInt else -1 System.setProperty("spark.serializer", "spark.KryoSerializer") System.setProperty("spark.locality.wait", "10000") val sc = new SparkContext(master, "ALS") @@ -397,7 +398,7 @@ object ALS { val fields = line.split(',') (fields(0).toInt, fields(1).toInt, fields(2).toDouble) } - val model = ALS.train(ratings, rank, iters) + val model = ALS.train(ratings, rank, iters, 0.01, blocks) model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } .saveAsTextFile(outputDir + "/userFeatures") model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } From d47c16f78d5cb935bd4022c9bed8376691371682 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 01:55:54 +0000 Subject: [PATCH 202/419] Add an option to disable reference tracking in Kryo --- core/src/main/scala/spark/KryoSerializer.scala | 4 ++++ docs/configuration.md | 12 +++++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index d723ab7b1e..c7dbcc6fbc 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -210,6 +210,10 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) } + + // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops + kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) + kryo } diff --git a/docs/configuration.md b/docs/configuration.md index 5a80510959..5c06897cae 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -197,9 +197,19 @@ Apart from these, the following properties are also available, and may be useful (e.g. map functions) reference large objects in the driver program. + + spark.kryo.referenceTracking + true + + Whether to track references to the same object when serializing data with Kryo, which is + necessary if your object graphs have loops and useful for efficiency if they contain multiple + copies of the same object. Can be disabled to improve performance if you know this is not the + case. + + spark.kryoserializer.buffer.mb - 32 + 2 Maximum object size to allow within Kryo (the library needs to create a buffer at least as large as the largest single object you'll serialize). Increase this if you get a "buffer limit From 4698a0d6886905ef21cbd52e108d0dcab3df12df Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 02:54:11 +0000 Subject: [PATCH 203/419] Shuffle ratings in a more efficient way at start of ALS --- .../scala/spark/mllib/recommendation/ALS.scala | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 2abaf2f2dd..4c18cbdc6b 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -6,8 +6,10 @@ import scala.util.Sorting import spark.{HashPartitioner, Partitioner, SparkContext, RDD} import spark.storage.StorageLevel +import spark.KryoRegistrator import spark.SparkContext._ +import com.esotericsoftware.kryo.Kryo import org.jblas.{DoubleMatrix, SimpleBlas, Solve} @@ -98,8 +100,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val partitioner = new HashPartitioner(numBlocks) - val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } - val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } + val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, Rating(u, p, r)) } + val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, Rating(p, u, r)) } val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) @@ -179,12 +181,12 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l * the users (or (blockId, (p, u, r)) for the products). We create these simultaneously to avoid * having to shuffle the (blockId, (u, p, r)) RDD twice, or to cache it. */ - private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, (Int, Int, Double))]) + private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)]) : (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) = { val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { - val ratings = elements.map{case (k, t) => Rating(t._1, t._2, t._3)}.toArray + val ratings = elements.map{_._2}.toArray val inLinkBlock = makeInLinkBlock(numBlocks, ratings) val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) @@ -383,6 +385,12 @@ object ALS { train(ratings, rank, iterations, 0.01, -1) } + private class ALSRegistrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[Rating]) + } + } + def main(args: Array[String]) { if (args.length != 5 && args.length != 6) { println("Usage: ALS []") @@ -392,6 +400,8 @@ object ALS { (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) val blocks = if (args.length == 6) args(5).toInt else -1 System.setProperty("spark.serializer", "spark.KryoSerializer") + System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) + System.setProperty("spark.kryo.referenceTracking", "false") System.setProperty("spark.locality.wait", "10000") val sc = new SparkContext(master, "ALS") val ratings = sc.textFile(ratingsFile).map { line => From b2aaa1199e7ecd8e1b2a9ddd8356b6393edafe6b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 11:44:42 -0700 Subject: [PATCH 204/419] Adds app name in HTML page titles on job web UI: fixes SPARK-806 --- core/src/main/scala/spark/ui/UIUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index 36d9c47245..fa46e2487d 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -31,7 +31,7 @@ private[spark] object UIUtils { - {title} + {sc.appName} - {title} From 0d78b6d9cd11fc12c546f25fa857ba8b285c062d Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 13:42:12 -0700 Subject: [PATCH 205/419] Links to job UI from standalone deploy cluster web UI: fixes SPARK-802 --- core/src/main/scala/spark/deploy/master/Master.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 4dd6c448a9..87a7791fbd 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -80,6 +80,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logInfo("Registering app " + description.name) val app = addApplication(description, sender) logInfo("Registered app " + description.name + " with ID " + app.id) + logInfo("Started App web UI at " + description.appUiUrl) waitingApps += app context.watch(sender) // This doesn't work with remote actors but helps for testing sender ! RegisteredApplication(app.id) From 39557112501901da7f9b4be6159a5a0be5511b42 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 15:47:21 -0700 Subject: [PATCH 206/419] Added field to master UI with link to job UI --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 7545ecf868..434f600e8e 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -26,8 +26,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User", - "State", "Duration") + val appHeaders = Seq("ID", "Job UI", "Description", "Cores", "Memory per Node", "Submit Time", + "User", "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) val completedApps = state.completedApps.sortBy(_.endTime).reverse @@ -103,6 +103,9 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.id} + + {app.appUiUrl} + {app.desc.name} {app.coresGranted} From fbf5aa761e40649d0488a8673d488882c8bdae48 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 15:50:03 -0700 Subject: [PATCH 207/419] Removed log message, added field in master UI to link to log UI --- core/src/main/scala/spark/deploy/master/Master.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 87a7791fbd..4dd6c448a9 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -80,7 +80,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logInfo("Registering app " + description.name) val app = addApplication(description, sender) logInfo("Registered app " + description.name + " with ID " + app.id) - logInfo("Started App web UI at " + description.appUiUrl) waitingApps += app context.watch(sender) // This doesn't work with remote actors but helps for testing sender ! RegisteredApplication(app.id) From 6dc7c9bfb17cefdf162e86c8b52a4ffc8b59efaf Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 16:33:50 -0700 Subject: [PATCH 208/419] Removed job UI column, linked description to job UI --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 434f600e8e..5e3c5e064f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -26,8 +26,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("ID", "Job UI", "Description", "Cores", "Memory per Node", "Submit Time", - "User", "State", "Duration") + val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User", + "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) val completedApps = state.completedApps.sortBy(_.endTime).reverse @@ -104,9 +104,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.id} - {app.appUiUrl} + {app.desc.name} - {app.desc.name} {app.coresGranted} From a96b4ef761aa80310b194176f41a088c5bf6274a Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Jul 2013 19:13:17 -0600 Subject: [PATCH 209/419] dding tgz option to make-distribution.sh --- make-distribution.sh | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index feb13d52f9..ef3d2529d0 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Script to create a binary distribution for easy deploys of Spark. # The distribution directory defaults to dist/ but can be overridden below. @@ -6,6 +6,10 @@ # so it is completely self contained. # It does not contain source or *.class files. # +# Arguments +# (none): Creates dist/ directory +# tgz: Additionally creates spark-$VERSION-bin.tar.gz +# # Recommended deploy/testing procedure (standalone mode): # 1) Rsync / deploy the dist/ dir to one host # 2) cd to deploy dir; ./bin/start-master.sh @@ -19,8 +23,14 @@ DISTDIR="$FWDIR/dist" # Get version from SBT export TERM=dumb # Prevents color codes in SBT output -VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2) -echo "Making distribution for Spark $VERSION in $DISTDIR..." +VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') + +if [ "$1" == "tgz" ]; then + echo "Making spark-$VERSION-bin.tar.gz" +else + echo "Making distribution for Spark $VERSION in $DISTDIR..." +fi + # Build fat JAR $FWDIR/sbt/sbt "repl/assembly" @@ -37,3 +47,11 @@ cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/conf" "$DISTDIR" cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" + + +if [ "$1" == "tgz" ]; then + TARDIR="$FWDIR/spark-$VERSION" + cp -r $DISTDIR $TARDIR + tar -zcf spark-$VERSION-bin.tar.gz -C $FWDIR spark-$VERSION + rm -rf $TARDIR +fi From 90b0142985ca2c6c76ded1a5c073774308c1a727 Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Jul 2013 19:13:39 -0600 Subject: [PATCH 210/419] adding files generated from make-distribution.sh to .gitignore --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index ae39c52b11..00fbff6a2c 100644 --- a/.gitignore +++ b/.gitignore @@ -38,3 +38,5 @@ dependency-reduced-pom.xml .ensime_lucene checkpoint derby.log +dist/ +spark-*-bin.tar.gz From 69316603d6bf11ecf1ea3dab63df178bad835e2d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 15 Jul 2013 22:50:11 -0700 Subject: [PATCH 211/419] Throw a more meaningful message when runJob is called to launch tasks on non-existent partitions. --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 9 +++++++++ core/src/test/scala/spark/RDDSuite.scala | 6 ++++++ 2 files changed, 15 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 3d3b9ea011..8173ef709d 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -251,6 +251,15 @@ class DAGScheduler( if (partitions.size == 0) { return } + + // Check to make sure we are not launching a task on a partition that does not exist. + val maxPartitions = finalRdd.partitions.length + partitions.find(p => p >= maxPartitions).foreach { p => + throw new IllegalArgumentException( + "Attempting to access a non-existent partition: " + p + ". " + + "Total number of partitions: " + maxPartitions) + } + val (toSubmit, waiter) = prepareJob( finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index aa3ee5f5ee..7f7d4c8211 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -302,4 +302,10 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") } } + + test("runJob on an invalid partition") { + intercept[IllegalArgumentException] { + sc.runJob(sc.parallelize(1 to 10, 2), {iter: Iterator[Int] => iter.size}, Seq(0, 1, 2), false) + } + } } From 2748e73eb9bb3f08b116c58ea404a6e76f9cbff9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 16:08:46 +0530 Subject: [PATCH 212/419] Dependency upgrade Akka 2.0.3 -> 2.0.5 --- project/SparkBuild.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 178a056de8..2510a614e8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -153,9 +153,9 @@ object SparkBuild extends Build { "org.ow2.asm" % "asm" % "4.0", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", - "com.typesafe.akka" % "akka-actor" % "2.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-remote" % "2.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-slf4j" % "2.0.3" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", @@ -235,7 +235,7 @@ object SparkBuild extends Build { "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-zeromq" % "2.0.3" excludeAll(excludeNetty) + "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty) ) ) ++ assemblySettings ++ extraAssemblySettings From f347cc3f659d4414a21de26feadcbe23a130e622 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 10:53:30 -0700 Subject: [PATCH 213/419] Fix deprecation warning and style issues --- core/src/test/scala/spark/FileServerSuite.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index 9c24ca430d..c7855a7bd3 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -35,26 +35,26 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal - }.collect + }.collect() assert(result.toSet === Set((1,200), (2,300), (3,500))) } test("Distributing files locally using URL as input") { // addFile("file:///....") sc = new SparkContext("local[4]", "test") - sc.addFile((new File(tmpFile.toString)).toURL.toString) + sc.addFile(new File(tmpFile.toString).toURI.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal - }.collect + }.collect() assert(result.toSet === Set((1,200), (2,300), (3,500))) } @@ -80,11 +80,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal - }.collect + }.collect() assert(result.toSet === Set((1,200), (2,300), (3,500))) } From 5c388808a81ebfb1fc23511882c18f9ae76ec509 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 11:01:14 -0700 Subject: [PATCH 214/419] SPARK-814: Result stages should be named after action --- .../scala/spark/scheduler/DAGScheduler.scala | 22 ++++++++++++------- .../main/scala/spark/scheduler/Stage.scala | 5 +++-- .../main/scala/spark/ui/jobs/IndexPage.scala | 2 +- .../spark/scheduler/JobLoggerSuite.scala | 4 ++-- 4 files changed, 20 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 3d3b9ea011..30a648f50b 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -150,7 +150,13 @@ class DAGScheduler( * as a result stage for the final RDD used directly in an action. The stage will also be given * the provided priority. */ - private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = { + private def newStage( + rdd: RDD[_], + shuffleDep: Option[ShuffleDependency[_,_]], + priority: Int, + callSite: Option[String] = None) + : Stage = + { if (shuffleDep != None) { // Kind of ugly: need to register RDDs with the cache and map output tracker here // since we can't do it in the RDD constructor because # of partitions is unknown @@ -158,7 +164,7 @@ class DAGScheduler( mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size) } val id = nextStageId.getAndIncrement() - val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) + val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority, callSite) idToStage(id) = stage stageToInfos(stage) = StageInfo(stage) stage @@ -286,12 +292,12 @@ class DAGScheduler( event match { case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) => val runId = nextRunId.getAndIncrement() - val finalStage = newStage(finalRDD, None, runId) + val finalStage = newStage(finalRDD, None, runId, Some(callSite)) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") - logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") + logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")") logInfo("Parents of final stage: " + finalStage.parents) logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { @@ -502,7 +508,7 @@ class DAGScheduler( case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0) case _ => "Unkown" } - logInfo("%s (%s) finished in %s s".format(stage, stage.origin, serviceTime)) + logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stage.completionTime = Some(System.currentTimeMillis) val stageComp = StageCompleted(stageToInfos(stage)) sparkListeners.foreach{_.onStageCompleted(stageComp)} @@ -568,7 +574,7 @@ class DAGScheduler( if (stage.outputLocs.count(_ == Nil) != 0) { // Some tasks had failed; let's resubmit this stage // TODO: Lower-level scheduler should also deal with this - logInfo("Resubmitting " + stage + " (" + stage.origin + + logInfo("Resubmitting " + stage + " (" + stage.name + ") because some of its tasks had failed: " + stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) submitStage(stage) @@ -600,7 +606,7 @@ class DAGScheduler( running -= failedStage failed += failedStage // TODO: Cancel running tasks in the stage - logInfo("Marking " + failedStage + " (" + failedStage.origin + + logInfo("Marking " + failedStage + " (" + failedStage.name + ") for resubmision due to a fetch failure") // Mark the map whose fetch failed as broken in the map stage val mapStage = shuffleToMapStage(shuffleId) @@ -608,7 +614,7 @@ class DAGScheduler( mapStage.removeOutputLoc(mapId, bmAddress) mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) } - logInfo("The failed fetch was from " + mapStage + " (" + mapStage.origin + + logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + "); marking it for resubmission") failed += mapStage // Remember that a fetch failed now; this is used to resubmit the broken diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 539cf8233b..4937eb3b88 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -24,7 +24,8 @@ private[spark] class Stage( val rdd: RDD[_], val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage val parents: List[Stage], - val priority: Int) + val priority: Int, + callSite: Option[String]) extends Logging { val isShuffleMap = shuffleDep != None @@ -85,7 +86,7 @@ private[spark] class Stage( return id } - def origin: String = rdd.origin + val name = callSite.getOrElse(rdd.origin) override def toString = "Stage " + id diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 7907ab3bc7..2df5f0192b 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -89,7 +89,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { {s.id} - {s.origin} + {s.name} {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 699901f1a1..2474d744b8 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -37,8 +37,8 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val parentRdd = makeRdd(4, Nil) val shuffleDep = new ShuffleDependency(parentRdd, null) val rootRdd = makeRdd(4, List(shuffleDep)) - val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) - val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) + val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID, None) + val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID, None) joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4)) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) From 00a14deb6d9b6d89e0ddbe1540636622b1dc4d16 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 11:52:20 -0700 Subject: [PATCH 215/419] Update to latest Scala Maven plugin and allow Zinc external compiler --- pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 63dd80b5fe..6fa1f1d4f1 100644 --- a/pom.xml +++ b/pom.xml @@ -319,7 +319,7 @@ net.alchim31.maven scala-maven-plugin - 3.1.0 + 3.1.5 scala-compile-first @@ -346,12 +346,14 @@ ${scala.version} incremental + true -unchecked -optimise -Xms64m + -Xms1024m -Xmx1024m -XX:PermSize=${PermGen} -XX:MaxPermSize=${MaxPermGen} From af3c9d50424602f3e5af1055e83e9badef0a1632 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 17:21:33 -0700 Subject: [PATCH 216/419] Add Apache license headers and LICENSE and NOTICE files --- LICENSE | 225 ++++++++++++++++-- NOTICE | 5 + bagel/pom.xml | 17 ++ bagel/src/main/scala/spark/bagel/Bagel.scala | 17 ++ .../spark/bagel/examples/PageRankUtils.scala | 17 ++ .../bagel/examples/WikipediaPageRank.scala | 17 ++ .../WikipediaPageRankStandalone.scala | 17 ++ bagel/src/test/resources/log4j.properties | 19 +- bagel/src/test/scala/bagel/BagelSuite.scala | 17 ++ bin/slaves.sh | 5 +- bin/spark-config.sh | 19 +- bin/spark-daemon.sh | 5 +- bin/spark-daemons.sh | 17 ++ bin/start-all.sh | 17 ++ bin/start-master.sh | 17 ++ bin/start-slave.sh | 17 ++ bin/start-slaves.sh | 17 ++ bin/stop-all.sh | 17 ++ bin/stop-master.sh | 17 ++ bin/stop-slaves.sh | 17 ++ core/pom.xml | 17 ++ .../hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 17 ++ .../yarn/ApplicationMasterArguments.scala | 17 ++ .../scala/spark/deploy/yarn/Client.scala | 17 ++ .../spark/deploy/yarn/ClientArguments.scala | 17 ++ .../spark/deploy/yarn/WorkerRunnable.scala | 17 ++ .../deploy/yarn/YarnAllocationHandler.scala | 17 ++ .../cluster/YarnClusterScheduler.scala | 17 ++ .../hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../java/spark/network/netty/FileClient.java | 17 ++ .../netty/FileClientChannelInitializer.java | 17 ++ .../network/netty/FileClientHandler.java | 17 ++ .../java/spark/network/netty/FileServer.java | 17 ++ .../netty/FileServerChannelInitializer.java | 17 ++ .../network/netty/FileServerHandler.java | 17 ++ .../spark/network/netty/PathResolver.java | 17 ++ core/src/main/scala/spark/Accumulators.scala | 17 ++ core/src/main/scala/spark/Aggregator.scala | 17 ++ .../spark/BlockStoreShuffleFetcher.scala | 17 ++ core/src/main/scala/spark/Cache.scala | 17 ++ core/src/main/scala/spark/CacheManager.scala | 17 ++ .../src/main/scala/spark/ClosureCleaner.scala | 17 ++ core/src/main/scala/spark/Dependency.scala | 17 ++ .../main/scala/spark/DoubleRDDFunctions.scala | 17 ++ .../scala/spark/FetchFailedException.scala | 17 ++ core/src/main/scala/spark/HadoopWriter.scala | 17 ++ .../src/main/scala/spark/HttpFileServer.scala | 19 +- core/src/main/scala/spark/HttpServer.scala | 17 ++ .../src/main/scala/spark/JavaSerializer.scala | 17 ++ .../src/main/scala/spark/KryoSerializer.scala | 17 ++ core/src/main/scala/spark/Logging.scala | 17 ++ .../main/scala/spark/MapOutputTracker.scala | 17 ++ .../main/scala/spark/PairRDDFunctions.scala | 17 ++ core/src/main/scala/spark/Partition.scala | 17 ++ core/src/main/scala/spark/Partitioner.scala | 17 ++ core/src/main/scala/spark/RDD.scala | 17 ++ .../main/scala/spark/RDDCheckpointData.scala | 17 ++ .../spark/SequenceFileRDDFunctions.scala | 17 ++ .../scala/spark/SerializableWritable.scala | 17 ++ .../src/main/scala/spark/ShuffleFetcher.scala | 17 ++ core/src/main/scala/spark/SizeEstimator.scala | 17 ++ .../main/scala/spark/SoftReferenceCache.scala | 17 ++ core/src/main/scala/spark/SparkContext.scala | 17 ++ core/src/main/scala/spark/SparkEnv.scala | 17 ++ .../src/main/scala/spark/SparkException.scala | 17 ++ core/src/main/scala/spark/SparkFiles.java | 17 ++ core/src/main/scala/spark/TaskContext.scala | 17 ++ core/src/main/scala/spark/TaskEndReason.scala | 17 ++ core/src/main/scala/spark/TaskState.scala | 17 ++ core/src/main/scala/spark/Utils.scala | 17 ++ .../scala/spark/api/java/JavaDoubleRDD.scala | 17 ++ .../scala/spark/api/java/JavaPairRDD.scala | 17 ++ .../main/scala/spark/api/java/JavaRDD.scala | 17 ++ .../scala/spark/api/java/JavaRDDLike.scala | 17 ++ .../spark/api/java/JavaSparkContext.scala | 17 ++ .../JavaSparkContextVarargsWorkaround.java | 17 ++ .../scala/spark/api/java/StorageLevels.java | 17 ++ .../java/function/DoubleFlatMapFunction.java | 17 ++ .../api/java/function/DoubleFunction.java | 17 ++ .../api/java/function/FlatMapFunction.scala | 17 ++ .../api/java/function/FlatMapFunction2.scala | 17 ++ .../spark/api/java/function/Function.java | 17 ++ .../spark/api/java/function/Function2.java | 17 ++ .../java/function/PairFlatMapFunction.java | 17 ++ .../spark/api/java/function/PairFunction.java | 17 ++ .../api/java/function/VoidFunction.scala | 19 +- .../api/java/function/WrappedFunction1.scala | 17 ++ .../api/java/function/WrappedFunction2.scala | 17 ++ .../spark/api/python/PythonPartitioner.scala | 17 ++ .../scala/spark/api/python/PythonRDD.scala | 17 ++ .../api/python/PythonWorkerFactory.scala | 17 ++ .../spark/broadcast/BitTorrentBroadcast.scala | 17 ++ .../scala/spark/broadcast/Broadcast.scala | 17 ++ .../spark/broadcast/BroadcastFactory.scala | 17 ++ .../scala/spark/broadcast/HttpBroadcast.scala | 17 ++ .../scala/spark/broadcast/MultiTracker.scala | 17 ++ .../scala/spark/broadcast/SourceInfo.scala | 17 ++ .../scala/spark/broadcast/TreeBroadcast.scala | 17 ++ .../spark/deploy/ApplicationDescription.scala | 17 ++ .../src/main/scala/spark/deploy/Command.scala | 17 ++ .../scala/spark/deploy/DeployMessage.scala | 17 ++ .../scala/spark/deploy/ExecutorState.scala | 17 ++ .../scala/spark/deploy/JsonProtocol.scala | 19 +- .../spark/deploy/LocalSparkCluster.scala | 17 ++ core/src/main/scala/spark/deploy/WebUI.scala | 17 ++ .../scala/spark/deploy/client/Client.scala | 17 ++ .../spark/deploy/client/ClientListener.scala | 17 ++ .../spark/deploy/client/TestClient.scala | 17 ++ .../spark/deploy/client/TestExecutor.scala | 17 ++ .../spark/deploy/master/ApplicationInfo.scala | 17 ++ .../deploy/master/ApplicationState.scala | 17 ++ .../spark/deploy/master/ExecutorInfo.scala | 17 ++ .../scala/spark/deploy/master/Master.scala | 17 ++ .../spark/deploy/master/MasterArguments.scala | 17 ++ .../spark/deploy/master/WorkerInfo.scala | 17 ++ .../spark/deploy/master/WorkerState.scala | 17 ++ .../deploy/master/ui/ApplicationPage.scala | 17 ++ .../spark/deploy/master/ui/IndexPage.scala | 17 ++ .../spark/deploy/master/ui/MasterWebUI.scala | 19 +- .../spark/deploy/worker/ExecutorRunner.scala | 17 ++ .../scala/spark/deploy/worker/Worker.scala | 17 ++ .../spark/deploy/worker/WorkerArguments.scala | 17 ++ .../spark/deploy/worker/ui/IndexPage.scala | 17 ++ .../spark/deploy/worker/ui/WorkerWebUI.scala | 17 ++ .../main/scala/spark/executor/Executor.scala | 17 ++ .../spark/executor/ExecutorBackend.scala | 17 ++ .../spark/executor/ExecutorExitCode.scala | 17 ++ .../executor/ExecutorURLClassLoader.scala | 17 ++ .../spark/executor/MesosExecutorBackend.scala | 17 ++ .../executor/StandaloneExecutorBackend.scala | 17 ++ .../scala/spark/executor/TaskMetrics.scala | 17 ++ .../scala/spark/network/BufferMessage.scala | 19 +- .../main/scala/spark/network/Connection.scala | 17 ++ .../spark/network/ConnectionManager.scala | 17 ++ .../spark/network/ConnectionManagerId.scala | 17 ++ .../spark/network/ConnectionManagerTest.scala | 17 ++ .../main/scala/spark/network/Message.scala | 17 ++ .../scala/spark/network/MessageChunk.scala | 17 ++ .../spark/network/MessageChunkHeader.scala | 17 ++ .../scala/spark/network/ReceiverTest.scala | 17 ++ .../main/scala/spark/network/SenderTest.scala | 17 ++ .../spark/network/netty/FileHeader.scala | 17 ++ .../spark/network/netty/ShuffleCopier.scala | 17 ++ .../spark/network/netty/ShuffleSender.scala | 17 ++ core/src/main/scala/spark/package.scala | 17 ++ .../partial/ApproximateActionListener.scala | 17 ++ .../spark/partial/ApproximateEvaluator.scala | 17 ++ .../scala/spark/partial/BoundedDouble.scala | 17 ++ .../scala/spark/partial/CountEvaluator.scala | 17 ++ .../spark/partial/GroupedCountEvaluator.scala | 17 ++ .../spark/partial/GroupedMeanEvaluator.scala | 17 ++ .../spark/partial/GroupedSumEvaluator.scala | 17 ++ .../scala/spark/partial/MeanEvaluator.scala | 17 ++ .../scala/spark/partial/PartialResult.scala | 17 ++ .../scala/spark/partial/StudentTCacher.scala | 17 ++ .../scala/spark/partial/SumEvaluator.scala | 17 ++ core/src/main/scala/spark/rdd/BlockRDD.scala | 17 ++ .../main/scala/spark/rdd/CartesianRDD.scala | 17 ++ .../main/scala/spark/rdd/CheckpointRDD.scala | 17 ++ .../main/scala/spark/rdd/CoGroupedRDD.scala | 17 ++ .../main/scala/spark/rdd/CoalescedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/EmptyRDD.scala | 17 ++ .../main/scala/spark/rdd/FilteredRDD.scala | 17 ++ .../main/scala/spark/rdd/FlatMappedRDD.scala | 17 ++ .../src/main/scala/spark/rdd/GlommedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/HadoopRDD.scala | 17 ++ core/src/main/scala/spark/rdd/JdbcRDD.scala | 17 ++ .../scala/spark/rdd/MapPartitionsRDD.scala | 17 ++ .../spark/rdd/MapPartitionsWithIndexRDD.scala | 17 ++ core/src/main/scala/spark/rdd/MappedRDD.scala | 17 ++ .../main/scala/spark/rdd/NewHadoopRDD.scala | 17 ++ .../spark/rdd/ParallelCollectionRDD.scala | 17 ++ .../scala/spark/rdd/PartitionPruningRDD.scala | 17 ++ core/src/main/scala/spark/rdd/PipedRDD.scala | 17 ++ .../src/main/scala/spark/rdd/SampledRDD.scala | 17 ++ .../main/scala/spark/rdd/ShuffledRDD.scala | 17 ++ .../main/scala/spark/rdd/SubtractedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/UnionRDD.scala | 17 ++ .../scala/spark/rdd/ZippedPartitionsRDD.scala | 17 ++ core/src/main/scala/spark/rdd/ZippedRDD.scala | 17 ++ .../scala/spark/scheduler/ActiveJob.scala | 17 ++ .../scala/spark/scheduler/DAGScheduler.scala | 17 ++ .../spark/scheduler/DAGSchedulerEvent.scala | 17 ++ .../spark/scheduler/InputFormatInfo.scala | 17 ++ .../scala/spark/scheduler/JobListener.scala | 17 ++ .../scala/spark/scheduler/JobLogger.scala | 17 ++ .../scala/spark/scheduler/JobResult.scala | 17 ++ .../scala/spark/scheduler/JobWaiter.scala | 17 ++ .../scala/spark/scheduler/MapStatus.scala | 17 ++ .../scala/spark/scheduler/ResultTask.scala | 17 ++ .../spark/scheduler/ShuffleMapTask.scala | 17 ++ .../scala/spark/scheduler/SparkListener.scala | 17 ++ .../scala/spark/scheduler/SplitInfo.scala | 17 ++ .../main/scala/spark/scheduler/Stage.scala | 17 ++ .../scala/spark/scheduler/StageInfo.scala | 19 +- .../src/main/scala/spark/scheduler/Task.scala | 17 ++ .../scala/spark/scheduler/TaskResult.scala | 17 ++ .../scala/spark/scheduler/TaskScheduler.scala | 17 ++ .../scheduler/TaskSchedulerListener.scala | 17 ++ .../main/scala/spark/scheduler/TaskSet.scala | 17 ++ .../scheduler/cluster/ClusterScheduler.scala | 17 ++ .../cluster/ClusterTaskSetManager.scala | 17 ++ .../cluster/ExecutorLossReason.scala | 17 ++ .../scala/spark/scheduler/cluster/Pool.scala | 17 ++ .../spark/scheduler/cluster/Schedulable.scala | 17 ++ .../cluster/SchedulableBuilder.scala | 17 ++ .../scheduler/cluster/SchedulerBackend.scala | 17 ++ .../cluster/SchedulingAlgorithm.scala | 17 ++ .../scheduler/cluster/SchedulingMode.scala | 17 ++ .../cluster/SparkDeploySchedulerBackend.scala | 17 ++ .../cluster/StandaloneClusterMessage.scala | 17 ++ .../cluster/StandaloneSchedulerBackend.scala | 17 ++ .../scheduler/cluster/TaskDescription.scala | 17 ++ .../spark/scheduler/cluster/TaskInfo.scala | 17 ++ .../scheduler/cluster/TaskSetManager.scala | 17 ++ .../spark/scheduler/cluster/WorkerOffer.scala | 17 ++ .../scheduler/local/LocalScheduler.scala | 17 ++ .../scheduler/local/LocalTaskSetManager.scala | 17 ++ .../mesos/CoarseMesosSchedulerBackend.scala | 17 ++ .../mesos/MesosSchedulerBackend.scala | 17 ++ .../scala/spark/serializer/Serializer.scala | 17 ++ .../spark/serializer/SerializerManager.scala | 17 ++ .../scala/spark/storage/BlockException.scala | 17 ++ .../spark/storage/BlockFetchTracker.scala | 17 ++ .../spark/storage/BlockFetcherIterator.scala | 17 ++ .../scala/spark/storage/BlockManager.scala | 17 ++ .../scala/spark/storage/BlockManagerId.scala | 17 ++ .../spark/storage/BlockManagerMaster.scala | 17 ++ .../storage/BlockManagerMasterActor.scala | 17 ++ .../spark/storage/BlockManagerMessages.scala | 17 ++ .../storage/BlockManagerSlaveActor.scala | 17 ++ .../spark/storage/BlockManagerWorker.scala | 17 ++ .../scala/spark/storage/BlockMessage.scala | 17 ++ .../spark/storage/BlockMessageArray.scala | 17 ++ .../spark/storage/BlockObjectWriter.scala | 17 ++ .../main/scala/spark/storage/BlockStore.scala | 17 ++ .../main/scala/spark/storage/DiskStore.scala | 17 ++ .../scala/spark/storage/MemoryStore.scala | 17 ++ .../main/scala/spark/storage/PutResult.scala | 17 ++ .../spark/storage/ShuffleBlockManager.scala | 17 ++ .../scala/spark/storage/StorageLevel.scala | 17 ++ .../scala/spark/storage/StorageUtils.scala | 17 ++ .../scala/spark/storage/ThreadingTest.scala | 17 ++ core/src/main/scala/spark/ui/JettyUtils.scala | 17 ++ core/src/main/scala/spark/ui/Page.scala | 19 +- core/src/main/scala/spark/ui/SparkUI.scala | 17 ++ core/src/main/scala/spark/ui/UIUtils.scala | 19 +- .../scala/spark/ui/UIWorkloadGenerator.scala | 17 ++ .../scala/spark/ui/env/EnvironmentUI.scala | 17 ++ .../main/scala/spark/ui/jobs/IndexPage.scala | 17 ++ .../scala/spark/ui/jobs/JobProgressUI.scala | 19 +- .../main/scala/spark/ui/jobs/StagePage.scala | 17 ++ .../spark/ui/storage/BlockManagerUI.scala | 17 ++ .../scala/spark/ui/storage/IndexPage.scala | 17 ++ .../main/scala/spark/ui/storage/RDDPage.scala | 17 ++ .../src/main/scala/spark/util/AkkaUtils.scala | 17 ++ .../spark/util/BoundedPriorityQueue.scala | 17 ++ .../spark/util/ByteBufferInputStream.scala | 17 ++ .../scala/spark/util/CompletionIterator.scala | 19 +- .../main/scala/spark/util/Distribution.scala | 19 +- .../main/scala/spark/util/IdGenerator.scala | 17 ++ core/src/main/scala/spark/util/IntParam.scala | 17 ++ .../main/scala/spark/util/MemoryParam.scala | 17 ++ .../scala/spark/util/MetadataCleaner.scala | 17 ++ .../main/scala/spark/util/NextIterator.scala | 19 +- .../spark/util/RateLimitedOutputStream.scala | 17 ++ .../scala/spark/util/SerializableBuffer.scala | 17 ++ .../main/scala/spark/util/StatCounter.scala | 17 ++ .../scala/spark/util/TimeStampedHashMap.scala | 17 ++ .../scala/spark/util/TimeStampedHashSet.scala | 17 ++ core/src/main/scala/spark/util/Vector.scala | 17 ++ core/src/test/resources/fairscheduler.xml | 17 ++ core/src/test/resources/log4j.properties | 19 +- .../test/scala/spark/AccumulatorSuite.scala | 17 ++ .../src/test/scala/spark/BroadcastSuite.scala | 17 ++ .../test/scala/spark/CheckpointSuite.scala | 17 ++ .../scala/spark/ClosureCleanerSuite.scala | 17 ++ .../test/scala/spark/DistributedSuite.scala | 17 ++ core/src/test/scala/spark/DriverSuite.scala | 17 ++ core/src/test/scala/spark/FailureSuite.scala | 17 ++ .../test/scala/spark/FileServerSuite.scala | 17 ++ core/src/test/scala/spark/FileSuite.scala | 17 ++ core/src/test/scala/spark/JavaAPISuite.java | 17 ++ .../scala/spark/KryoSerializerSuite.scala | 17 ++ .../test/scala/spark/LocalSparkContext.scala | 17 ++ .../scala/spark/MapOutputTrackerSuite.scala | 17 ++ .../scala/spark/PairRDDFunctionsSuite.scala | 17 ++ .../test/scala/spark/PartitioningSuite.scala | 17 ++ core/src/test/scala/spark/PipedRDDSuite.scala | 17 ++ core/src/test/scala/spark/RDDSuite.scala | 17 ++ .../test/scala/spark/SharedSparkContext.scala | 17 ++ .../test/scala/spark/ShuffleNettySuite.scala | 17 ++ core/src/test/scala/spark/ShuffleSuite.scala | 17 ++ .../test/scala/spark/SizeEstimatorSuite.scala | 17 ++ core/src/test/scala/spark/SortingSuite.scala | 17 ++ .../src/test/scala/spark/ThreadingSuite.scala | 17 ++ .../src/test/scala/spark/UnpersistSuite.scala | 17 ++ core/src/test/scala/spark/UtilsSuite.scala | 17 ++ .../scala/spark/ZippedPartitionsSuite.scala | 17 ++ .../test/scala/spark/rdd/JdbcRDDSuite.scala | 17 ++ .../rdd/ParallelCollectionSplitSuite.scala | 17 ++ .../scheduler/ClusterSchedulerSuite.scala | 17 ++ .../spark/scheduler/DAGSchedulerSuite.scala | 17 ++ .../spark/scheduler/JobLoggerSuite.scala | 17 ++ .../spark/scheduler/LocalSchedulerSuite.scala | 17 ++ .../spark/scheduler/SparkListenerSuite.scala | 17 ++ .../spark/scheduler/TaskContextSuite.scala | 17 ++ .../spark/storage/BlockManagerSuite.scala | 17 ++ core/src/test/scala/spark/ui/UISuite.scala | 17 ++ .../scala/spark/util/DistributionSuite.scala | 17 ++ .../scala/spark/util/NextIteratorSuite.scala | 17 ++ .../util/RateLimitedOutputStreamSuite.scala | 17 ++ docs/_plugins/copy_api_dirs.rb | 17 ++ ec2/spark-ec2 | 2 + ec2/spark_ec2.py | 2 + examples/pom.xml | 17 ++ .../main/java/spark/examples/JavaHdfsLR.java | 17 ++ .../main/java/spark/examples/JavaKMeans.java | 17 ++ .../java/spark/examples/JavaLogQuery.java | 17 ++ .../main/java/spark/examples/JavaSparkPi.java | 17 ++ .../src/main/java/spark/examples/JavaTC.java | 17 ++ .../java/spark/examples/JavaWordCount.java | 17 ++ .../examples/JavaFlumeEventCount.java | 17 ++ .../examples/JavaNetworkWordCount.java | 17 ++ .../streaming/examples/JavaQueueStream.java | 17 ++ .../scala/spark/examples/BroadcastTest.scala | 17 ++ .../scala/spark/examples/CassandraTest.scala | 17 ++ .../examples/ExceptionHandlingTest.scala | 17 ++ .../scala/spark/examples/GroupByTest.scala | 17 ++ .../main/scala/spark/examples/HBaseTest.scala | 19 +- .../main/scala/spark/examples/HdfsTest.scala | 17 ++ .../main/scala/spark/examples/LocalALS.scala | 17 ++ .../scala/spark/examples/LocalFileLR.scala | 17 ++ .../scala/spark/examples/LocalKMeans.scala | 17 ++ .../main/scala/spark/examples/LocalLR.scala | 17 ++ .../main/scala/spark/examples/LocalPi.scala | 17 ++ .../main/scala/spark/examples/LogQuery.scala | 17 ++ .../spark/examples/MultiBroadcastTest.scala | 17 ++ .../examples/SimpleSkewedGroupByTest.scala | 17 ++ .../spark/examples/SkewedGroupByTest.scala | 17 ++ .../main/scala/spark/examples/SparkALS.scala | 17 ++ .../scala/spark/examples/SparkHdfsLR.scala | 17 ++ .../scala/spark/examples/SparkKMeans.scala | 17 ++ .../main/scala/spark/examples/SparkLR.scala | 17 ++ .../main/scala/spark/examples/SparkPi.scala | 17 ++ .../main/scala/spark/examples/SparkTC.scala | 17 ++ .../streaming/examples/ActorWordCount.scala | 17 ++ .../streaming/examples/FlumeEventCount.scala | 17 ++ .../streaming/examples/HdfsWordCount.scala | 17 ++ .../streaming/examples/KafkaWordCount.scala | 17 ++ .../streaming/examples/NetworkWordCount.scala | 17 ++ .../streaming/examples/QueueStream.scala | 17 ++ .../streaming/examples/RawNetworkGrep.scala | 17 ++ .../examples/StatefulNetworkWordCount.scala | 17 ++ .../examples/TwitterAlgebirdCMS.scala | 17 ++ .../examples/TwitterAlgebirdHLL.scala | 17 ++ .../examples/TwitterPopularTags.scala | 17 ++ .../streaming/examples/ZeroMQWordCount.scala | 17 ++ .../clickstream/PageViewGenerator.scala | 17 ++ .../examples/clickstream/PageViewStream.scala | 17 ++ make-distribution.sh | 19 ++ .../scala/spark/mllib/clustering/KMeans.scala | 17 ++ .../spark/mllib/clustering/KMeansModel.scala | 17 ++ .../spark/mllib/clustering/LocalKMeans.scala | 17 ++ .../spark/mllib/optimization/Gradient.scala | 17 ++ .../mllib/optimization/GradientDescent.scala | 17 ++ .../spark/mllib/optimization/Updater.scala | 17 ++ .../spark/mllib/recommendation/ALS.scala | 17 ++ .../MatrixFactorizationModel.scala | 17 ++ .../mllib/regression/LogisticRegression.scala | 17 ++ .../LogisticRegressionGenerator.scala | 17 ++ .../spark/mllib/regression/Regression.scala | 17 ++ .../mllib/regression/RidgeRegression.scala | 17 ++ .../regression/RidgeRegressionGenerator.scala | 17 ++ .../main/scala/spark/mllib/util/MLUtils.scala | 17 ++ mllib/src/test/resources/log4j.properties | 19 +- .../spark/mllib/clustering/KMeansSuite.scala | 17 ++ .../spark/mllib/recommendation/ALSSuite.scala | 17 ++ .../regression/LogisticRegressionSuite.scala | 17 ++ .../regression/RidgeRegressionSuite.scala | 17 ++ pom.xml | 17 ++ project/SparkBuild.scala | 16 ++ project/build.properties | 17 ++ pyspark | 17 ++ python/epydoc.conf | 17 ++ python/examples/als.py | 17 ++ python/examples/kmeans.py | 17 ++ python/examples/logistic_regression.py | 17 ++ python/examples/pi.py | 17 ++ python/examples/transitive_closure.py | 17 ++ python/examples/wordcount.py | 17 ++ python/pyspark/accumulators.py | 17 ++ python/pyspark/broadcast.py | 17 ++ python/pyspark/context.py | 17 ++ python/pyspark/daemon.py | 17 ++ python/pyspark/files.py | 17 ++ python/pyspark/java_gateway.py | 17 ++ python/pyspark/rdd.py | 17 ++ python/pyspark/serializers.py | 17 ++ python/pyspark/shell.py | 17 ++ python/pyspark/tests.py | 17 ++ python/pyspark/worker.py | 17 ++ python/run-tests | 20 +- repl-bin/pom.xml | 17 ++ repl-bin/src/deb/bin/run | 19 +- repl-bin/src/deb/bin/spark-executor | 17 ++ repl-bin/src/deb/bin/spark-shell | 17 ++ repl/pom.xml | 17 ++ .../spark/repl/ExecutorClassLoader.scala | 17 ++ repl/src/main/scala/spark/repl/Main.scala | 17 ++ repl/src/test/resources/log4j.properties | 19 +- .../src/test/scala/spark/repl/ReplSuite.scala | 17 ++ run | 17 ++ run.cmd | 18 ++ run2.cmd | 17 ++ sbt/sbt | 20 ++ sbt/sbt.cmd | 20 ++ spark-executor | 18 ++ spark-shell | 19 ++ spark-shell.cmd | 18 ++ streaming/pom.xml | 17 ++ .../scala/spark/streaming/Checkpoint.scala | 17 ++ .../main/scala/spark/streaming/DStream.scala | 17 ++ .../streaming/DStreamCheckpointData.scala | 17 ++ .../scala/spark/streaming/DStreamGraph.scala | 17 ++ .../main/scala/spark/streaming/Duration.scala | 17 ++ .../main/scala/spark/streaming/Interval.scala | 17 ++ .../src/main/scala/spark/streaming/Job.scala | 17 ++ .../scala/spark/streaming/JobManager.scala | 17 ++ .../spark/streaming/NetworkInputTracker.scala | 17 ++ .../streaming/PairDStreamFunctions.scala | 17 ++ .../scala/spark/streaming/Scheduler.scala | 17 ++ .../spark/streaming/StreamingContext.scala | 17 ++ .../src/main/scala/spark/streaming/Time.scala | 19 +- .../streaming/api/java/JavaDStream.scala | 19 +- .../streaming/api/java/JavaDStreamLike.scala | 19 +- .../streaming/api/java/JavaPairDStream.scala | 17 ++ .../api/java/JavaStreamingContext.scala | 17 ++ .../streaming/dstream/CoGroupedDStream.scala | 17 ++ .../dstream/ConstantInputDStream.scala | 19 +- .../streaming/dstream/FileInputDStream.scala | 17 ++ .../streaming/dstream/FilteredDStream.scala | 17 ++ .../dstream/FlatMapValuedDStream.scala | 17 ++ .../streaming/dstream/FlatMappedDStream.scala | 17 ++ .../streaming/dstream/FlumeInputDStream.scala | 17 ++ .../streaming/dstream/ForEachDStream.scala | 17 ++ .../streaming/dstream/GlommedDStream.scala | 17 ++ .../streaming/dstream/InputDStream.scala | 17 ++ .../streaming/dstream/KafkaInputDStream.scala | 17 ++ .../dstream/MapPartitionedDStream.scala | 17 ++ .../streaming/dstream/MapValuedDStream.scala | 17 ++ .../streaming/dstream/MappedDStream.scala | 17 ++ .../dstream/NetworkInputDStream.scala | 17 ++ .../dstream/PluggableInputDStream.scala | 17 ++ .../streaming/dstream/QueueInputDStream.scala | 17 ++ .../streaming/dstream/RawInputDStream.scala | 17 ++ .../dstream/ReducedWindowedDStream.scala | 17 ++ .../streaming/dstream/ShuffledDStream.scala | 17 ++ .../dstream/SocketInputDStream.scala | 17 ++ .../streaming/dstream/StateDStream.scala | 17 ++ .../dstream/TransformedDStream.scala | 17 ++ .../dstream/TwitterInputDStream.scala | 17 ++ .../streaming/dstream/UnionDStream.scala | 17 ++ .../streaming/dstream/WindowedDStream.scala | 17 ++ .../streaming/receivers/ActorReceiver.scala | 17 ++ .../streaming/receivers/ZeroMQReceiver.scala | 17 ++ .../scala/spark/streaming/util/Clock.scala | 17 ++ .../streaming/util/MasterFailureTest.scala | 17 ++ .../spark/streaming/util/RawTextHelper.scala | 17 ++ .../spark/streaming/util/RawTextSender.scala | 17 ++ .../spark/streaming/util/RecurringTimer.scala | 17 ++ .../java/spark/streaming/JavaAPISuite.java | 17 ++ .../java/spark/streaming/JavaTestUtils.scala | 19 +- streaming/src/test/resources/log4j.properties | 19 +- .../streaming/BasicOperationsSuite.scala | 17 ++ .../spark/streaming/CheckpointSuite.scala | 17 ++ .../scala/spark/streaming/FailureSuite.scala | 17 ++ .../spark/streaming/InputStreamsSuite.scala | 17 ++ .../scala/spark/streaming/TestSuiteBase.scala | 17 ++ .../streaming/WindowOperationsSuite.scala | 17 ++ 488 files changed, 8446 insertions(+), 57 deletions(-) create mode 100644 NOTICE diff --git a/LICENSE b/LICENSE index d17afa1fc6..d645695673 100644 --- a/LICENSE +++ b/LICENSE @@ -1,27 +1,202 @@ -Copyright (c) 2010, Regents of the University of California. -All rights reserved. -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions -are met: - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - * Neither the name of the University of California, Berkeley nor the - names of its contributors may be used to endorse or promote - products derived from this software without specific prior written - permission. + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED -TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/NOTICE b/NOTICE new file mode 100644 index 0000000000..7cbb114b2a --- /dev/null +++ b/NOTICE @@ -0,0 +1,5 @@ +Apache Spark +Copyright 2013 The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/bagel/pom.xml b/bagel/pom.xml index b83a0ef6c0..60bbc49e6c 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/spark/bagel/Bagel.scala index 5ecdd7d004..80c8d53d2b 100644 --- a/bagel/src/main/scala/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/spark/bagel/Bagel.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel import spark._ diff --git a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala index b97d786ed4..de65e27fe0 100644 --- a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala +++ b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel.examples import spark._ diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala index bc32663e0f..a0c5ac9c18 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala +++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel.examples import spark._ diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala index 9d9d80d809..3c54a85f42 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala +++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel.examples import spark._ diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 83d05cab2f..5cdcf35b23 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file bagel/target/unit-tests.log +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file bagel/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala index a09c978068..ef2d57fbd0 100644 --- a/bagel/src/test/scala/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/bagel/BagelSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.bagel import org.scalatest.{FunSuite, Assertions, BeforeAndAfter} diff --git a/bin/slaves.sh b/bin/slaves.sh index 030581e058..c8fb5ca473 100755 --- a/bin/slaves.sh +++ b/bin/slaves.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# This Spark deploy script is a modified version of the Apache Hadoop deploy -# script, available under the Apache 2 license: # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -10,13 +8,14 @@ # (the "License"); you may not use this file except in compliance with # the License. You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# http://www.apache.org/licenses/LICENSE-2.0 # # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +# # Run a shell command on all slave hosts. # diff --git a/bin/spark-config.sh b/bin/spark-config.sh index d4b6558866..cd2c7b7b0d 100755 --- a/bin/spark-config.sh +++ b/bin/spark-config.sh @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # included in all the spark scripts with source command # should not be executable directly # also should not be passed any arguments, since we need original $* @@ -16,4 +33,4 @@ this="$config_bin/$script" export SPARK_PREFIX=`dirname "$this"`/.. export SPARK_HOME=${SPARK_PREFIX} -export SPARK_CONF_DIR="$SPARK_HOME/conf" \ No newline at end of file +export SPARK_CONF_DIR="$SPARK_HOME/conf" diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh index 8ee3ec481f..a5b88ca785 100755 --- a/bin/spark-daemon.sh +++ b/bin/spark-daemon.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# This Spark deploy script is a modified version of the Apache Hadoop deploy -# script, available under the Apache 2 license: # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -10,13 +8,14 @@ # (the "License"); you may not use this file except in compliance with # the License. You may obtain a copy of the License at # -# http://www.apache.org/licenses/LICENSE-2.0 +# http://www.apache.org/licenses/LICENSE-2.0 # # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +# # Runs a Spark command as a daemon. # diff --git a/bin/spark-daemons.sh b/bin/spark-daemons.sh index 0619097e4d..354eb905a1 100755 --- a/bin/spark-daemons.sh +++ b/bin/spark-daemons.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Run a Spark command on all slave hosts. usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..." diff --git a/bin/start-all.sh b/bin/start-all.sh index b9891ad2f6..0182f1ab24 100755 --- a/bin/start-all.sh +++ b/bin/start-all.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Start all spark daemons. # Starts the master on this node. # Starts a worker on each node specified in conf/slaves diff --git a/bin/start-master.sh b/bin/start-master.sh index 83a3e1f3dc..2288fb19d7 100755 --- a/bin/start-master.sh +++ b/bin/start-master.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Starts the master on the machine this script is executed on. bin=`dirname "$0"` diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 1082c09eb1..d6db16882d 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash + # +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Usage: start-slave.sh # where is like "spark://localhost:7077" diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh index 4e05224190..dad7c3df76 100755 --- a/bin/start-slaves.sh +++ b/bin/start-slaves.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + bin=`dirname "$0"` bin=`cd "$bin"; pwd` diff --git a/bin/stop-all.sh b/bin/stop-all.sh index d352f6f631..a043ac0095 100755 --- a/bin/stop-all.sh +++ b/bin/stop-all.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Start all spark daemons. # Run this on the master nde diff --git a/bin/stop-master.sh b/bin/stop-master.sh index 172ee5891d..31a610bf9d 100755 --- a/bin/stop-master.sh +++ b/bin/stop-master.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Starts the master on the machine this script is executed on. bin=`dirname "$0"` diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh index fbfc594472..8e056f23d4 100755 --- a/bin/stop-slaves.sh +++ b/bin/stop-slaves.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Starts the master on the machine this script is executed on. bin=`dirname "$0"` diff --git a/core/pom.xml b/core/pom.xml index dbb2da9a9c..6329b2fbd8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index f286f2cf9c..25386b2796 100644 --- a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapred trait HadoopMapRedUtil { diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 264d421d14..b1002e0cac 100644 --- a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapreduce import org.apache.hadoop.conf.Configuration diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index f1c86de4cc..df55be1254 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index 875c0a220b..0f972b7a0b 100644 --- a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -1,4 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapred import org.apache.hadoop.mapreduce.TaskType diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 8bc6fb6dea..1a7cdf4788 100644 --- a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapreduce import org.apache.hadoop.conf.Configuration diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala index 301a57fffa..6122fdced0 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import collection.mutable.HashMap diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 6a0617cc06..1b06169739 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import java.net.Socket diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala index 1b00208511..8de44b1f66 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import spark.util.IntParam diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala index 514c17f241..8bcbfc2735 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import java.net.{InetSocketAddress, URI} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala index 07e7edea36..67aff03781 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import spark.util.MemoryParam diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala index cc6f3344a1..f458f2f6a1 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import java.net.URI diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala index 61dd72a651..b0af8baf08 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.yarn import spark.{Logging, Utils} diff --git a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala index ed732d36bf..307d96111c 100644 --- a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark._ diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index a0652d7fc7..4b3d84670c 100644 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapred trait HadoopMapRedUtil { diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 7fdbe322fd..aa3b1ed3a5 100644 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapreduce import org.apache.hadoop.conf.Configuration diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index f1c86de4cc..df55be1254 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index a4bb4bc701..0625a6d502 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import io.netty.bootstrap.Bootstrap; diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java index af25baf641..05ad4b61d7 100644 --- a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import io.netty.buffer.BufType; diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java index 9fc9449827..e8cd9801f6 100644 --- a/core/src/main/java/spark/network/netty/FileClientHandler.java +++ b/core/src/main/java/spark/network/netty/FileClientHandler.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import io.netty.buffer.ByteBuf; diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index dd3a557ae5..9f009a61d5 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import java.net.InetSocketAddress; diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java index 8f1f5c65cd..50c57a81a3 100644 --- a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import io.netty.channel.ChannelInitializer; diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/spark/network/netty/FileServerHandler.java index a78eddb1b5..176ba8da49 100644 --- a/core/src/main/java/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/spark/network/netty/FileServerHandler.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; import java.io.File; diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/spark/network/netty/PathResolver.java index 302411672c..f446c55b19 100755 --- a/core/src/main/java/spark/network/netty/PathResolver.java +++ b/core/src/main/java/spark/network/netty/PathResolver.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty; diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala index 57c6df35be..6ff92ce833 100644 --- a/core/src/main/scala/spark/Accumulators.scala +++ b/core/src/main/scala/spark/Accumulators.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/spark/Aggregator.scala index df8ce9c054..136b4da61e 100644 --- a/core/src/main/scala/spark/Aggregator.scala +++ b/core/src/main/scala/spark/Aggregator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 3239f4c385..8f6953b1f5 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala index 20d677a854..b0c83ce59d 100644 --- a/core/src/main/scala/spark/Cache.scala +++ b/core/src/main/scala/spark/Cache.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.util.concurrent.atomic.AtomicInteger diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala index f7a2b7e802..81314805a9 100644 --- a/core/src/main/scala/spark/CacheManager.scala +++ b/core/src/main/scala/spark/CacheManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.{ArrayBuffer, HashSet} diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala index d5e7132ff9..8b39241095 100644 --- a/core/src/main/scala/spark/ClosureCleaner.scala +++ b/core/src/main/scala/spark/ClosureCleaner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.lang.reflect.Field diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index 2af44aa383..d17e70a4fa 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark /** diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala index 178d31a73b..93ef097702 100644 --- a/core/src/main/scala/spark/DoubleRDDFunctions.scala +++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import spark.partial.BoundedDouble diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/spark/FetchFailedException.scala index 40b0193f19..a2dae6cae9 100644 --- a/core/src/main/scala/spark/FetchFailedException.scala +++ b/core/src/main/scala/spark/FetchFailedException.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import spark.storage.BlockManagerId diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala index 5e8396edb9..b1fe0075a3 100644 --- a/core/src/main/scala/spark/HadoopWriter.scala +++ b/core/src/main/scala/spark/HadoopWriter.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.mapred import org.apache.hadoop.fs.FileSystem diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/spark/HttpFileServer.scala index 00901d95e2..a13a7a2859 100644 --- a/core/src/main/scala/spark/HttpFileServer.scala +++ b/core/src/main/scala/spark/HttpFileServer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.{File} @@ -42,4 +59,4 @@ private[spark] class HttpFileServer extends Logging { return dir + "/" + file.getName } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/HttpServer.scala b/core/src/main/scala/spark/HttpServer.scala index 4e0507c080..c9dffbc631 100644 --- a/core/src/main/scala/spark/HttpServer.scala +++ b/core/src/main/scala/spark/HttpServer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.File diff --git a/core/src/main/scala/spark/JavaSerializer.scala b/core/src/main/scala/spark/JavaSerializer.scala index b04a27d073..04c5f44e6b 100644 --- a/core/src/main/scala/spark/JavaSerializer.scala +++ b/core/src/main/scala/spark/JavaSerializer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index c7dbcc6fbc..ee37da7948 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala index 0fc8c31463..79b0362830 100644 --- a/core/src/main/scala/spark/Logging.scala +++ b/core/src/main/scala/spark/Logging.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.slf4j.Logger diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index fde597ffd1..2c417e31db 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 8b313c645f..6b0cc2fbf1 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/Partition.scala b/core/src/main/scala/spark/Partition.scala index e384308ef6..2a4edcec98 100644 --- a/core/src/main/scala/spark/Partition.scala +++ b/core/src/main/scala/spark/Partition.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark /** diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 6f8cd17c88..660af70d52 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark /** diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 8aa77266bc..ca7cdd622a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.util.Random diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala index 57e0405fb4..b615f820eb 100644 --- a/core/src/main/scala/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/spark/RDDCheckpointData.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.apache.hadoop.fs.Path diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 2911f9036e..9f30b7f22f 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.EOFException diff --git a/core/src/main/scala/spark/SerializableWritable.scala b/core/src/main/scala/spark/SerializableWritable.scala index 8306fbf570..0236611ef9 100644 --- a/core/src/main/scala/spark/SerializableWritable.scala +++ b/core/src/main/scala/spark/SerializableWritable.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 9513a00126..dcced035e7 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import spark.executor.TaskMetrics diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala index f8a4c4e489..6cc57566d7 100644 --- a/core/src/main/scala/spark/SizeEstimator.scala +++ b/core/src/main/scala/spark/SizeEstimator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.lang.reflect.Field diff --git a/core/src/main/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala index 3dd0a4b1f9..f41a379582 100644 --- a/core/src/main/scala/spark/SoftReferenceCache.scala +++ b/core/src/main/scala/spark/SoftReferenceCache.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import com.google.common.collect.MapMaker diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 43e6af2351..46b9935cb7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index ec59b4f48f..f2bdc11bdb 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import collection.mutable diff --git a/core/src/main/scala/spark/SparkException.scala b/core/src/main/scala/spark/SparkException.scala index aa7a16d7dd..b7045eea63 100644 --- a/core/src/main/scala/spark/SparkException.scala +++ b/core/src/main/scala/spark/SparkException.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark class SparkException(message: String, cause: Throwable) diff --git a/core/src/main/scala/spark/SparkFiles.java b/core/src/main/scala/spark/SparkFiles.java index 566aec622c..f9b3f7965e 100644 --- a/core/src/main/scala/spark/SparkFiles.java +++ b/core/src/main/scala/spark/SparkFiles.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark; import java.io.File; diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index dd0609026a..b79f4ca813 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import executor.TaskMetrics diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index bb75ec208c..3ad665da34 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import spark.executor.TaskMetrics diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala index 78eb33a628..9df7d8277b 100644 --- a/core/src/main/scala/spark/TaskState.scala +++ b/core/src/main/scala/spark/TaskState.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.apache.mesos.Protos.{TaskState => MesosTaskState} diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index a36186bf8a..e6a96a5ec1 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io._ diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 16692c0440..392556f261 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import spark.RDD diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 76051597b6..ccc511dc5f 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import java.util.{List => JList} diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index 626b499454..c0bf2cf568 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import spark._ diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 27f40ecdfd..21b5abf053 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import java.util.{List => JList, Comparator} diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index 5f18b1e15b..fe182e7ab6 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java import java.util.{Map => JMap} diff --git a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java index 97344e73da..42b1de01b1 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java +++ b/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java; import java.util.Arrays; diff --git a/core/src/main/scala/spark/api/java/StorageLevels.java b/core/src/main/scala/spark/api/java/StorageLevels.java index 5e5845ac3a..f385636e83 100644 --- a/core/src/main/scala/spark/api/java/StorageLevels.java +++ b/core/src/main/scala/spark/api/java/StorageLevels.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java; import spark.storage.StorageLevel; diff --git a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java index 3a8192be3a..8bc88d757f 100644 --- a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java +++ b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; diff --git a/core/src/main/scala/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFunction.java index c6ef76d088..1aa1e5dae0 100644 --- a/core/src/main/scala/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/scala/spark/api/java/function/DoubleFunction.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala index e027cdacd3..9eb0cfe3f9 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function /** diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala index 6044043add..dda98710c2 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function /** diff --git a/core/src/main/scala/spark/api/java/function/Function.java b/core/src/main/scala/spark/api/java/function/Function.java index dae8295f21..2a2ea0aacf 100644 --- a/core/src/main/scala/spark/api/java/function/Function.java +++ b/core/src/main/scala/spark/api/java/function/Function.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; import scala.reflect.ClassManifest; diff --git a/core/src/main/scala/spark/api/java/function/Function2.java b/core/src/main/scala/spark/api/java/function/Function2.java index 69bf12c8c9..952d31ece4 100644 --- a/core/src/main/scala/spark/api/java/function/Function2.java +++ b/core/src/main/scala/spark/api/java/function/Function2.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; import scala.reflect.ClassManifest; diff --git a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java index b3cc4df6aa..4aad602da3 100644 --- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; import scala.Tuple2; diff --git a/core/src/main/scala/spark/api/java/function/PairFunction.java b/core/src/main/scala/spark/api/java/function/PairFunction.java index 9fc6df4b88..ccfe64ecf1 100644 --- a/core/src/main/scala/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFunction.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function; import scala.Tuple2; diff --git a/core/src/main/scala/spark/api/java/function/VoidFunction.scala b/core/src/main/scala/spark/api/java/function/VoidFunction.scala index b0096cf2bf..f6fc0b0f7d 100644 --- a/core/src/main/scala/spark/api/java/function/VoidFunction.scala +++ b/core/src/main/scala/spark/api/java/function/VoidFunction.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function /** @@ -13,4 +30,4 @@ abstract class VoidFunction[T] extends Serializable { // return Unit), so it is implicitly converted to a Function1[T, Unit]: object VoidFunction { implicit def toFunction[T](f: VoidFunction[T]) : Function1[T, Unit] = ((x : T) => f.call(x)) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala index 923f5cdf4f..1758a38c4e 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala +++ b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function import scala.runtime.AbstractFunction1 diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala index 2c6e9b1571..b093567d2c 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala +++ b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.java.function import scala.runtime.AbstractFunction2 diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala index d618c098c2..31a719fbff 100644 --- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.python import spark.Partitioner diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 31d8ea89d4..af10822dbd 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.python import java.io._ diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 85d1dfeac8..078ad45ce8 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.api.python import java.io.{DataInputStream, IOException} diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala index adcb2d2415..6f7d385379 100644 --- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.io._ diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala index 415bde5d67..aba56a60ca 100644 --- a/core/src/main/scala/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/spark/broadcast/Broadcast.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.io._ diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala index 5c6184c3c7..d33d95c7d9 100644 --- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast /** diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala index 7e30b8f7d2..c565876950 100644 --- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/spark/broadcast/MultiTracker.scala index 3fd77af73f..7855d44e9b 100644 --- a/core/src/main/scala/spark/broadcast/MultiTracker.scala +++ b/core/src/main/scala/spark/broadcast/MultiTracker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.io._ diff --git a/core/src/main/scala/spark/broadcast/SourceInfo.scala b/core/src/main/scala/spark/broadcast/SourceInfo.scala index c79bb93c38..b17ae63b5c 100644 --- a/core/src/main/scala/spark/broadcast/SourceInfo.scala +++ b/core/src/main/scala/spark/broadcast/SourceInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.util.BitSet diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala index c55c476117..ea1e9a12c1 100644 --- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.broadcast import java.io._ diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index 02193c7008..a8b22fbef8 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy private[spark] class ApplicationDescription( diff --git a/core/src/main/scala/spark/deploy/Command.scala b/core/src/main/scala/spark/deploy/Command.scala index 577101e3c3..bad629e965 100644 --- a/core/src/main/scala/spark/deploy/Command.scala +++ b/core/src/main/scala/spark/deploy/Command.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import scala.collection.Map diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 51274acb1e..e1f8aff6f5 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import spark.deploy.ExecutorState.ExecutorState diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/spark/deploy/ExecutorState.scala index 5dc0c54552..08c9a3b725 100644 --- a/core/src/main/scala/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/spark/deploy/ExecutorState.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy private[spark] object ExecutorState diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 335aaeb769..64f89623e1 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import master.{ApplicationInfo, WorkerInfo} @@ -62,4 +79,4 @@ private[spark] object JsonProtocol { ("executors" -> obj.executors.toList.map(writeExecutorRunner)) ~ ("finishedexecutors" -> obj.finishedExecutors.toList.map(writeExecutorRunner)) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 939f26b6f4..6b8e9f27af 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} diff --git a/core/src/main/scala/spark/deploy/WebUI.scala b/core/src/main/scala/spark/deploy/WebUI.scala index 844c4142c7..8ea7792ef4 100644 --- a/core/src/main/scala/spark/deploy/WebUI.scala +++ b/core/src/main/scala/spark/deploy/WebUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy import java.text.SimpleDateFormat diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 4af44f9c16..edefa0292d 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.client import spark.deploy._ diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala index e8c4083f9d..064024455e 100644 --- a/core/src/main/scala/spark/deploy/client/ClientListener.scala +++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.client /** diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index f195082808..4f4daa141a 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.client import spark.util.AkkaUtils diff --git a/core/src/main/scala/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/spark/deploy/client/TestExecutor.scala index 0e46db2272..8a22b6b89f 100644 --- a/core/src/main/scala/spark/deploy/client/TestExecutor.scala +++ b/core/src/main/scala/spark/deploy/client/TestExecutor.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.client private[spark] object TestExecutor { diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 785c16e2be..15ff919738 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import spark.deploy.ApplicationDescription diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/spark/deploy/master/ApplicationState.scala index 15016b388d..94f0ad8bae 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationState.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master private[spark] object ApplicationState diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala index 48e6055fb5..99b60f7d09 100644 --- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import spark.deploy.ExecutorState diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 4dd6c448a9..e5a7a87e2e 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import akka.actor._ diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala index 3d28ecabb4..d0ec3d5ea0 100644 --- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import spark.util.IntParam diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala index 0c08c5f417..4135cfeb28 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master import akka.actor.ActorRef diff --git a/core/src/main/scala/spark/deploy/master/WorkerState.scala b/core/src/main/scala/spark/deploy/master/WorkerState.scala index 0bf35014c8..3e50b7748d 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerState.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 8553377d8f..32264af393 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master.ui import akka.dispatch.Await diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 5e3c5e064f..b05197c1b9 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master.ui import akka.dispatch.Await diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index 82bc6961e2..04b32c7968 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.master.ui import akka.actor.ActorRef @@ -56,4 +73,4 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten private[spark] object MasterWebUI { val STATIC_RESOURCE_DIR = "spark/ui/static" val DEFAULT_PORT = "8080" -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 5d3d54c65e..8b51ff1c3a 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker import java.io._ diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index f20ea42d7f..0bd88ea253 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker import scala.collection.mutable.{ArrayBuffer, HashMap} diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala index 2b96611ee3..9fcd3260ca 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker import spark.util.IntParam diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index c515f2e238..7548a26c2e 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker.ui import akka.dispatch.Await diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index ccd55c1ce4..61d4cd6d99 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.deploy.worker.ui import akka.actor.ActorRef diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 8360547a74..2e81151882 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.io.{File, FileOutputStream} diff --git a/core/src/main/scala/spark/executor/ExecutorBackend.scala b/core/src/main/scala/spark/executor/ExecutorBackend.scala index e97e509700..33a6f8a824 100644 --- a/core/src/main/scala/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/ExecutorBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/spark/executor/ExecutorExitCode.scala index fd76029cb3..64b9fb88f8 100644 --- a/core/src/main/scala/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/spark/executor/ExecutorExitCode.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor /** diff --git a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala index 5beb4d049e..09d12fb65b 100644 --- a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.net.{URLClassLoader, URL} diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 10f3531df0..4961c42fad 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index ebe2ac68d8..f4003da732 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 1dc13754f9..3151627839 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.executor class TaskMetrics extends Serializable { diff --git a/core/src/main/scala/spark/network/BufferMessage.scala b/core/src/main/scala/spark/network/BufferMessage.scala index 7b0e489a6c..e566aeac13 100644 --- a/core/src/main/scala/spark/network/BufferMessage.scala +++ b/core/src/main/scala/spark/network/BufferMessage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer @@ -91,4 +108,4 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: "BufferMessage(id = " + id + ", size = " + size + ")" } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index 6e28f677a3..b66c00b58c 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import spark._ diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 624a094856..6c4e7dc03e 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import spark._ diff --git a/core/src/main/scala/spark/network/ConnectionManagerId.scala b/core/src/main/scala/spark/network/ConnectionManagerId.scala index b554e84251..9d5c518293 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerId.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.net.InetSocketAddress diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala index 533e4610f3..9e3827aaf5 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import spark._ diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala index d4f03610eb..a25457ea35 100644 --- a/core/src/main/scala/spark/network/Message.scala +++ b/core/src/main/scala/spark/network/Message.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/MessageChunk.scala b/core/src/main/scala/spark/network/MessageChunk.scala index aaf9204d0e..784db5ab62 100644 --- a/core/src/main/scala/spark/network/MessageChunk.scala +++ b/core/src/main/scala/spark/network/MessageChunk.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/MessageChunkHeader.scala b/core/src/main/scala/spark/network/MessageChunkHeader.scala index 3693d509d6..18d0cbcc14 100644 --- a/core/src/main/scala/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/spark/network/MessageChunkHeader.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.net.InetAddress diff --git a/core/src/main/scala/spark/network/ReceiverTest.scala b/core/src/main/scala/spark/network/ReceiverTest.scala index a174d5f403..2bbc736f40 100644 --- a/core/src/main/scala/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/spark/network/ReceiverTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/SenderTest.scala b/core/src/main/scala/spark/network/SenderTest.scala index a4ff69e4d2..542c54c36b 100644 --- a/core/src/main/scala/spark/network/SenderTest.scala +++ b/core/src/main/scala/spark/network/SenderTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/netty/FileHeader.scala b/core/src/main/scala/spark/network/netty/FileHeader.scala index aed4254234..bf46d32aa3 100644 --- a/core/src/main/scala/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/spark/network/netty/FileHeader.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty import io.netty.buffer._ diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index 8d5194a737..b01f6369f6 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty import java.util.concurrent.Executors diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala index d6fa4b1e80..cdf88b03a0 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.network.netty import java.io.File diff --git a/core/src/main/scala/spark/package.scala b/core/src/main/scala/spark/package.scala index 389ec4da3e..b244bfbf06 100644 --- a/core/src/main/scala/spark/package.scala +++ b/core/src/main/scala/spark/package.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + /** * Core Spark functionality. [[spark.SparkContext]] serves as the main entry point to Spark, while * [[spark.RDD]] is the data type representing a distributed collection, and provides most diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala index de2dce161a..691d939150 100644 --- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import spark._ diff --git a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala index 75713b2eaa..5eae144dfb 100644 --- a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala +++ b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial /** diff --git a/core/src/main/scala/spark/partial/BoundedDouble.scala b/core/src/main/scala/spark/partial/BoundedDouble.scala index 463c33d6e2..8bdbe6c012 100644 --- a/core/src/main/scala/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/spark/partial/BoundedDouble.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial /** diff --git a/core/src/main/scala/spark/partial/CountEvaluator.scala b/core/src/main/scala/spark/partial/CountEvaluator.scala index daf2c5170c..6aa92094eb 100644 --- a/core/src/main/scala/spark/partial/CountEvaluator.scala +++ b/core/src/main/scala/spark/partial/CountEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala index 01fbb8a11b..ebe2e5a1e3 100644 --- a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala index c622df5220..2dadbbd5fb 100644 --- a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala index 20fa55cff2..ae2b63f7cb 100644 --- a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/partial/MeanEvaluator.scala b/core/src/main/scala/spark/partial/MeanEvaluator.scala index 762c85400d..5ddcad7075 100644 --- a/core/src/main/scala/spark/partial/MeanEvaluator.scala +++ b/core/src/main/scala/spark/partial/MeanEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/PartialResult.scala b/core/src/main/scala/spark/partial/PartialResult.scala index 200ed4ea1e..922a9f9bc6 100644 --- a/core/src/main/scala/spark/partial/PartialResult.scala +++ b/core/src/main/scala/spark/partial/PartialResult.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial class PartialResult[R](initialVal: R, isFinal: Boolean) { diff --git a/core/src/main/scala/spark/partial/StudentTCacher.scala b/core/src/main/scala/spark/partial/StudentTCacher.scala index 443abba5cd..f3bb987d46 100644 --- a/core/src/main/scala/spark/partial/StudentTCacher.scala +++ b/core/src/main/scala/spark/partial/StudentTCacher.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/SumEvaluator.scala b/core/src/main/scala/spark/partial/SumEvaluator.scala index 58fb60f441..4083abef03 100644 --- a/core/src/main/scala/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/spark/partial/SumEvaluator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 719d4bf03e..0ebb722d73 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 38600b8be4..150e5bca29 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.io.{ObjectOutputStream, IOException} diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 377b1bdbe0..6794e0e201 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark._ diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 8966f9f86e..c540cd36eb 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.io.{ObjectOutputStream, IOException} diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 6d862c0c28..2b5bf18541 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/spark/rdd/EmptyRDD.scala index e4dd3a7fa7..d7d4db5d30 100644 --- a/core/src/main/scala/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/spark/rdd/EmptyRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala index c84ec39d21..783508cfd1 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{OneToOneDependency, RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala index 8ebc778925..ed75eac3ff 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala index e16c7ba881..1573f8a289 100644 --- a/core/src/main/scala/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 07c103503c..d0fdeb741e 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.io.EOFException diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index a50f407737..59132437d2 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.sql.{Connection, ResultSet} diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala index d283c5b2bb..af8f0a112f 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala index afb7504ba1..3b4e9518fd 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala index af07311b6d..8b411dd85d 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 901d01ef30..17fe805fd4 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.text.SimpleDateFormat diff --git a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala index 07585a88ce..16ba0c26f8 100644 --- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import scala.collection.immutable.NumericRange diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 41ff62dd22..191cfde565 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index c0baf43d43..2cefdc78b0 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.io.PrintWriter diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index 243673f151..574c9b141d 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.util.Random diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index c7d1926b83..0137f80953 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 8a9efc5da2..0402b9f250 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 2c52a67e22..2776826f18 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index b234428ab2..6a4fa13ad6 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index f728e93d24..b1c43b3195 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala index 105eaecb22..71cc94edb6 100644 --- a/core/src/main/scala/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.TaskContext diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 64ed91f5a0..29e879aa42 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import cluster.TaskInfo diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index acad915f13..506c87f65b 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala index 17d0ea4f80..65f8c3200e 100644 --- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.Logging diff --git a/core/src/main/scala/spark/scheduler/JobListener.scala b/core/src/main/scala/spark/scheduler/JobListener.scala index f46b9d551d..af108b8fec 100644 --- a/core/src/main/scala/spark/scheduler/JobListener.scala +++ b/core/src/main/scala/spark/scheduler/JobListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler /** diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 6a9d52f356..85b5ddd4a8 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.io.PrintWriter diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala index a0fdf391e6..a61b335152 100644 --- a/core/src/main/scala/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/spark/scheduler/JobResult.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler /** diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala index 6ff2e29434..69cd161c1f 100644 --- a/core/src/main/scala/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/spark/scheduler/MapStatus.scala index 203abb917b..2f6a68ee85 100644 --- a/core/src/main/scala/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/spark/scheduler/MapStatus.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.storage.BlockManagerId diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 83166bce22..361b1e6b91 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark._ diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 95647389c3..1c25605f75 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.io._ diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 8de3aa91a4..4fb1c5d42d 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/SplitInfo.scala b/core/src/main/scala/spark/scheduler/SplitInfo.scala index 6abfb7a1f7..4e3661ec5d 100644 --- a/core/src/main/scala/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/spark/scheduler/SplitInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 4937eb3b88..5428daeb94 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.net.URI diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 8d83ff10c4..c4026f995a 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.scheduler.cluster.TaskInfo @@ -9,4 +26,4 @@ case class StageInfo( val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]() ) { override def toString = stage.rdd.toString -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index a6462c6968..50768d43e0 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.serializer.SerializerInstance diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 6de0aa7adf..dc0621ea7b 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.io._ diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala index 7787b54762..5188308006 100644 --- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler /** diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index b75d3736cf..245e7ccb52 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/spark/scheduler/TaskSet.scala index e4b5fcaedb..dc3550dd0b 100644 --- a/core/src/main/scala/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/spark/scheduler/TaskSet.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 3a0c29b27f..7c10074dc7 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import java.lang.{Boolean => JBoolean} diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 327d6797ae..3d06520675 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} diff --git a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala index 8bf838209f..8825f2dd24 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.executor.ExecutorExitCode diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index 941ba7a3f1..83708f07e1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 2dd9c0564f..f557b142c4 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index 18cc15c2a5..95554023c0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import java.io.{File, FileInputStream, FileOutputStream} diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala index 8844057a5c..4431744ec3 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.{SparkContext, Utils} diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index f33310a34a..69e0ac2a6b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster /** diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 6e0c6793e0..4b3e3e50e1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster object SchedulingMode extends Enumeration("FAIR","FIFO"){ diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 170ede0f44..55d6c0a47e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.{Utils, Logging, SparkContext} diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index 3335294844..ac9e5ef94d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.TaskState.TaskState diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 004592a540..03a64e0192 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala index b41e951be9..761fdf6919 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index 718f26bfbd..a1ebd48b01 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import spark.Utils diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index b4dd75d90f..07c3ddcc7e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala index c47824315c..06d1203f70 100644 --- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala +++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.cluster /** diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index b000e328e6..1f73cb99a7 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.local import java.io.File diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index f12fec41d5..e662ad6709 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.local import java.io.File diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index f4a2994b6d..7bc6040544 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.mesos import com.google.protobuf.ByteString diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index e83368b98d..75b8268b55 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler.mesos import com.google.protobuf.ByteString diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala index 2ad73b711d..dc94d42bb6 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/spark/serializer/Serializer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.serializer import java.io.{EOFException, InputStream, OutputStream} diff --git a/core/src/main/scala/spark/serializer/SerializerManager.scala b/core/src/main/scala/spark/serializer/SerializerManager.scala index 60b2aac797..b7b24705a2 100644 --- a/core/src/main/scala/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/spark/serializer/SerializerManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.serializer import java.util.concurrent.ConcurrentHashMap diff --git a/core/src/main/scala/spark/storage/BlockException.scala b/core/src/main/scala/spark/storage/BlockException.scala index f275d476df..8ebfaf3cbf 100644 --- a/core/src/main/scala/spark/storage/BlockException.scala +++ b/core/src/main/scala/spark/storage/BlockException.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage private[spark] diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala index 0718156b1b..265e554ad8 100644 --- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala +++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage private[spark] trait BlockFetchTracker { diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index bec876213e..1965c5bc19 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9b39d3aadf..e4ffa57ad2 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{InputStream, OutputStream} diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index 1e557d6148..b36a6176c0 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 58888b1ebb..3186f7c85b 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import akka.actor.ActorRef diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 0d4384ba1f..244000d952 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 0010726c8d..01de4ccb8f 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{Externalizable, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala index b264d1deb5..45cffad810 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import akka.actor.Actor diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala index 3057ade233..39064bce92 100644 --- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala index 30d7500e01..ab72dbb62b 100644 --- a/core/src/main/scala/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/spark/storage/BlockMessage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala index ee0c5ff9a2..b0229d6124 100644 --- a/core/src/main/scala/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala index 42e2b07d5c..01ed6e8c1f 100644 --- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockStore.scala b/core/src/main/scala/spark/storage/BlockStore.scala index 8188d3595e..c8db0022b0 100644 --- a/core/src/main/scala/spark/storage/BlockStore.scala +++ b/core/src/main/scala/spark/storage/BlockStore.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index da859eebcb..3495d653bd 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala index eba5ee507f..b5a86b85a7 100644 --- a/core/src/main/scala/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/spark/storage/MemoryStore.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.util.LinkedHashMap diff --git a/core/src/main/scala/spark/storage/PutResult.scala b/core/src/main/scala/spark/storage/PutResult.scala index 76f236057b..3a0974fe15 100644 --- a/core/src/main/scala/spark/storage/PutResult.scala +++ b/core/src/main/scala/spark/storage/PutResult.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 44638e0c2d..8a7a6f9ed3 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import spark.serializer.Serializer diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index cc0c354e7e..f52650988c 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index aca16f533a..2aeed4ea3c 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import spark.{Utils, SparkContext} diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index 5c406e68cb..b3ab1ff4b4 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import akka.actor._ diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index bc6f9c10d5..ca6088ad93 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import annotation.tailrec diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala index ed8f91842c..a31e750d06 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/spark/ui/Page.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui -private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } \ No newline at end of file +private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 874e5ba8ec..9396f22063 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index fa46e2487d..b1d11954dd 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import scala.xml.Node @@ -115,4 +132,4 @@ private[spark] object UIUtils { } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 8bbc6ce88e..a80e2d7002 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import scala.util.Random diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index 6b8b9f05bb..5ae7935ed4 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.env import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 2df5f0192b..f31af3cda6 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.jobs import java.util.Date diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 84730cc091..44dcf82d11 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.jobs import akka.util.Duration @@ -124,4 +141,4 @@ private[spark] class JobProgressListener extends SparkListener { } return false // No tasks have finished for this stage } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 51b82b6a8c..292966f23a 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.jobs import java.util.Date diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index 3d67888e2c..49ed069c75 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.storage import akka.util.Duration diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index d284134391..4e0360d19a 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.storage import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 428db6fa95..003be54ad8 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui.storage import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 18fc9673bf..9233277bdb 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import akka.actor.{ActorSystem, ExtendedActorSystem} diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala index 4bc5db8bb7..0575497f5d 100644 --- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.io.Serializable diff --git a/core/src/main/scala/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/spark/util/ByteBufferInputStream.scala index d7e67497fe..47a28e2f76 100644 --- a/core/src/main/scala/spark/util/ByteBufferInputStream.scala +++ b/core/src/main/scala/spark/util/ByteBufferInputStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.io.InputStream diff --git a/core/src/main/scala/spark/util/CompletionIterator.scala b/core/src/main/scala/spark/util/CompletionIterator.scala index 8139183780..210450892b 100644 --- a/core/src/main/scala/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/spark/util/CompletionIterator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util /** @@ -22,4 +39,4 @@ object CompletionIterator { def completion() = completionFunction } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala index 24738b4307..5d4d7a6c50 100644 --- a/core/src/main/scala/spark/util/Distribution.scala +++ b/core/src/main/scala/spark/util/Distribution.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.io.PrintStream @@ -62,4 +79,4 @@ object Distribution { quantiles.foreach{q => out.print(q + "\t")} out.println } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/util/IdGenerator.scala b/core/src/main/scala/spark/util/IdGenerator.scala index b6e309fe1a..3422280559 100644 --- a/core/src/main/scala/spark/util/IdGenerator.scala +++ b/core/src/main/scala/spark/util/IdGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.util.concurrent.atomic.AtomicInteger diff --git a/core/src/main/scala/spark/util/IntParam.scala b/core/src/main/scala/spark/util/IntParam.scala index 0427646747..daf0d58fa2 100644 --- a/core/src/main/scala/spark/util/IntParam.scala +++ b/core/src/main/scala/spark/util/IntParam.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util /** diff --git a/core/src/main/scala/spark/util/MemoryParam.scala b/core/src/main/scala/spark/util/MemoryParam.scala index 3726738842..298562323a 100644 --- a/core/src/main/scala/spark/util/MemoryParam.scala +++ b/core/src/main/scala/spark/util/MemoryParam.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import spark.Utils diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index dafa906712..92909e0959 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors} diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/spark/util/NextIterator.scala index 48b5018ddd..22163ece8d 100644 --- a/core/src/main/scala/spark/util/NextIterator.scala +++ b/core/src/main/scala/spark/util/NextIterator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util /** Provides a basic/boilerplate Iterator implementation. */ @@ -68,4 +85,4 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { gotNext = false nextValue } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala index e3f00ea8c7..00f782bbe7 100644 --- a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala +++ b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import scala.annotation.tailrec diff --git a/core/src/main/scala/spark/util/SerializableBuffer.scala b/core/src/main/scala/spark/util/SerializableBuffer.scala index 09d588fe1c..7e6842628a 100644 --- a/core/src/main/scala/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/spark/util/SerializableBuffer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/spark/util/StatCounter.scala index 2b980340b7..76358d4151 100644 --- a/core/src/main/scala/spark/util/StatCounter.scala +++ b/core/src/main/scala/spark/util/StatCounter.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util /** diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index 92dfaa6e6f..cc7909194a 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import java.util.concurrent.ConcurrentHashMap diff --git a/core/src/main/scala/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/spark/util/TimeStampedHashSet.scala index 5f1cc93752..41e3fd8cba 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashSet.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import scala.collection.mutable.Set diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/spark/util/Vector.scala index 835822edb2..ed49386f18 100644 --- a/core/src/main/scala/spark/util/Vector.scala +++ b/core/src/main/scala/spark/util/Vector.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util class Vector(val elements: Array[Double]) extends Serializable { diff --git a/core/src/test/resources/fairscheduler.xml b/core/src/test/resources/fairscheduler.xml index 6e573b1883..996ffb1864 100644 --- a/core/src/test/resources/fairscheduler.xml +++ b/core/src/test/resources/fairscheduler.xml @@ -1,4 +1,21 @@ + + 2 diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index d05cf3dec1..f6fef03689 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file core/target/unit-tests.log +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file core/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index ac8ae7d308..0af175f316 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/BroadcastSuite.scala b/core/src/test/scala/spark/BroadcastSuite.scala index 362a31fb0d..785721ece8 100644 --- a/core/src/test/scala/spark/BroadcastSuite.scala +++ b/core/src/test/scala/spark/BroadcastSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 28a7b21b92..a84c89e3c9 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/ClosureCleanerSuite.scala b/core/src/test/scala/spark/ClosureCleanerSuite.scala index b2d0dd4627..7d2831e19c 100644 --- a/core/src/test/scala/spark/ClosureCleanerSuite.scala +++ b/core/src/test/scala/spark/ClosureCleanerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.NotSerializableException diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 0024ede828..e11efe459c 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import network.ConnectionManagerId diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index 31c3dd75fb..ed16b9d8ef 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.File diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index 8c1445a465..6c847b8fef 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index c7855a7bd3..242ae971f8 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import com.google.common.io.Files diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index e61ff7793d..1e2c257c4b 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.io.{FileWriter, PrintWriter, File} diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index d306124fca..aaf03e683b 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark; import java.io.File; diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index 327e2ff848..c3323dcbb3 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala index bd184222ed..ddc212d290 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.Suite diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index 6e585e1c3a..ce6cec0451 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala index 682d2745bf..b102eaf4e6 100644 --- a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.ArrayBuffer diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index 99e433e3bd..b1e0b2b4d0 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index d263bb00e9..35c04710a3 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 7f7d4c8211..cbddf4e523 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.HashMap diff --git a/core/src/test/scala/spark/SharedSparkContext.scala b/core/src/test/scala/spark/SharedSparkContext.scala index 1da79f9824..70c24515be 100644 --- a/core/src/test/scala/spark/SharedSparkContext.scala +++ b/core/src/test/scala/spark/SharedSparkContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.Suite diff --git a/core/src/test/scala/spark/ShuffleNettySuite.scala b/core/src/test/scala/spark/ShuffleNettySuite.scala index bfaffa953e..6bad6c1d13 100644 --- a/core/src/test/scala/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/spark/ShuffleNettySuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.BeforeAndAfterAll diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 950218fa28..3a56c26bef 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable.ArrayBuffer diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala index b5c8525f91..1ef812dfbd 100644 --- a/core/src/test/scala/spark/SizeEstimatorSuite.scala +++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala index f7bf207c68..b933c4aab8 100644 --- a/core/src/test/scala/spark/SortingSuite.scala +++ b/core/src/test/scala/spark/SortingSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/ThreadingSuite.scala b/core/src/test/scala/spark/ThreadingSuite.scala index ff315b6693..f2acd0bd3c 100644 --- a/core/src/test/scala/spark/ThreadingSuite.scala +++ b/core/src/test/scala/spark/ThreadingSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import java.util.concurrent.Semaphore diff --git a/core/src/test/scala/spark/UnpersistSuite.scala b/core/src/test/scala/spark/UnpersistSuite.scala index 94776e7572..93977d16f4 100644 --- a/core/src/test/scala/spark/UnpersistSuite.scala +++ b/core/src/test/scala/spark/UnpersistSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 1e1260f606..31c3b25c50 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import com.google.common.base.Charsets diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala index 96cb295f45..5e6d7b09d8 100644 --- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/spark/ZippedPartitionsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.immutable.NumericRange diff --git a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala index 6afb0fa9bc..dc8ca941c1 100644 --- a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import org.scalatest.{ BeforeAndAfter, FunSuite } diff --git a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala index d27a2538e4..d1276d541f 100644 --- a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.rdd import scala.collection.immutable.NumericRange diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 8e1ad27e14..8f81d0b6ee 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 30e6fef950..f802b66cf1 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import scala.collection.mutable.{Map, HashMap} diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 2474d744b8..0f855c38da 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import java.util.Properties diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala index 8bd813fd14..14bb58731b 100644 --- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala index 48aa67c543..392d67d67b 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala index 647bcaf860..95a6eee2fc 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index b9d5f9668e..b719d65342 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.storage import java.nio.ByteBuffer diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index fc0c160720..56c1fed6ad 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.ui import scala.util.{Failure, Success, Try} diff --git a/core/src/test/scala/spark/util/DistributionSuite.scala b/core/src/test/scala/spark/util/DistributionSuite.scala index cc6249b1dd..6578b55e82 100644 --- a/core/src/test/scala/spark/util/DistributionSuite.scala +++ b/core/src/test/scala/spark/util/DistributionSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/util/NextIteratorSuite.scala b/core/src/test/scala/spark/util/NextIteratorSuite.scala index ed5b36da73..fdbd43d941 100644 --- a/core/src/test/scala/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/spark/util/NextIteratorSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala index 794063fb6d..4c0044202f 100644 --- a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala +++ b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.util import org.scalatest.FunSuite diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index c10ae595de..45ef4bba82 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + require 'fileutils' include FileUtils diff --git a/ec2/spark-ec2 b/ec2/spark-ec2 index 2714f19ba3..454057aa0d 100755 --- a/ec2/spark-ec2 +++ b/ec2/spark-ec2 @@ -1,5 +1,6 @@ #!/bin/sh +# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -15,6 +16,7 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +# cd "`dirname $0`" PYTHONPATH="./third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" python ./spark_ec2.py $@ diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 7affe6fffc..2ec3c007fb 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -1,6 +1,7 @@ #!/usr/bin/env python # -*- coding: utf-8 -*- +# # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file # distributed with this work for additional information @@ -16,6 +17,7 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. +# from __future__ import with_statement diff --git a/examples/pom.xml b/examples/pom.xml index 1976765c3d..7a8d08fade 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/spark/examples/JavaHdfsLR.java index 8b0a9b6808..9485e0cfa9 100644 --- a/examples/src/main/java/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/spark/examples/JavaHdfsLR.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import spark.api.java.JavaRDD; diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/spark/examples/JavaKMeans.java index 626034eb0d..2d34776177 100644 --- a/examples/src/main/java/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/spark/examples/JavaKMeans.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import scala.Tuple2; diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/spark/examples/JavaLogQuery.java index 6b22e7120c..d22684d980 100644 --- a/examples/src/main/java/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/spark/examples/JavaLogQuery.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import com.google.common.collect.Lists; diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/spark/examples/JavaSparkPi.java index a15a967de8..d5f42fbb38 100644 --- a/examples/src/main/java/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/spark/examples/JavaSparkPi.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import spark.api.java.JavaRDD; diff --git a/examples/src/main/java/spark/examples/JavaTC.java b/examples/src/main/java/spark/examples/JavaTC.java index b319bdab44..559d7f9e53 100644 --- a/examples/src/main/java/spark/examples/JavaTC.java +++ b/examples/src/main/java/spark/examples/JavaTC.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import scala.Tuple2; diff --git a/examples/src/main/java/spark/examples/JavaWordCount.java b/examples/src/main/java/spark/examples/JavaWordCount.java index 9d4c7a252d..1af370c1c3 100644 --- a/examples/src/main/java/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/spark/examples/JavaWordCount.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples; import scala.Tuple2; diff --git a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java index e24c6ddaa7..096a9ae219 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples; import spark.api.java.function.Function; diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java index 3e57580fd4..c54d3f3d59 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples; import com.google.common.collect.Lists; diff --git a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java index 15b82c8da1..1f4a991542 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples; import com.google.common.collect.Lists; diff --git a/examples/src/main/scala/spark/examples/BroadcastTest.scala b/examples/src/main/scala/spark/examples/BroadcastTest.scala index ba59be1687..911490cb6c 100644 --- a/examples/src/main/scala/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/BroadcastTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 0fe1833e83..104bfd5204 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import org.apache.hadoop.mapreduce.Job diff --git a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala index 21a90f2e5a..67ddaec8d2 100644 --- a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/GroupByTest.scala b/examples/src/main/scala/spark/examples/GroupByTest.scala index a6603653f1..5cee413615 100644 --- a/examples/src/main/scala/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/spark/examples/GroupByTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala index 6e910154d4..4dd6c243ac 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark._ @@ -32,4 +49,4 @@ object HBaseTest { System.exit(0) } -} \ No newline at end of file +} diff --git a/examples/src/main/scala/spark/examples/HdfsTest.scala b/examples/src/main/scala/spark/examples/HdfsTest.scala index dd61c467f7..23258336e2 100644 --- a/examples/src/main/scala/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/spark/examples/HdfsTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark._ diff --git a/examples/src/main/scala/spark/examples/LocalALS.scala b/examples/src/main/scala/spark/examples/LocalALS.scala index 2de810e062..7a449a9d72 100644 --- a/examples/src/main/scala/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/spark/examples/LocalALS.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import scala.math.sqrt diff --git a/examples/src/main/scala/spark/examples/LocalFileLR.scala b/examples/src/main/scala/spark/examples/LocalFileLR.scala index f958ef9f72..c1f8d32aa8 100644 --- a/examples/src/main/scala/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/spark/examples/LocalFileLR.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/spark/examples/LocalKMeans.scala index 4849f216fb..0a0bc6f476 100644 --- a/examples/src/main/scala/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/spark/examples/LocalKMeans.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/LocalLR.scala b/examples/src/main/scala/spark/examples/LocalLR.scala index cd73f553d6..ab99bf1fbe 100644 --- a/examples/src/main/scala/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/spark/examples/LocalLR.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/LocalPi.scala b/examples/src/main/scala/spark/examples/LocalPi.scala index 9457472f2d..ccd69695df 100644 --- a/examples/src/main/scala/spark/examples/LocalPi.scala +++ b/examples/src/main/scala/spark/examples/LocalPi.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import scala.math.random diff --git a/examples/src/main/scala/spark/examples/LogQuery.scala b/examples/src/main/scala/spark/examples/LogQuery.scala index 6497596d35..e815ececf7 100644 --- a/examples/src/main/scala/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/spark/examples/LogQuery.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala index a0aaf60918..d0b1cf06e5 100644 --- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala index 461b84a2c6..d197bbaf7c 100644 --- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala index 435675f9de..4641b82444 100644 --- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/SparkALS.scala b/examples/src/main/scala/spark/examples/SparkALS.scala index 8fb3b0fb2a..ba0dfd8f9b 100644 --- a/examples/src/main/scala/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/spark/examples/SparkALS.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import scala.math.sqrt diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala index 3d080a0257..ef6e09a8e8 100644 --- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/SparkKMeans.scala b/examples/src/main/scala/spark/examples/SparkKMeans.scala index 4161c59fea..38ed3b149a 100644 --- a/examples/src/main/scala/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/spark/examples/SparkKMeans.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/SparkLR.scala b/examples/src/main/scala/spark/examples/SparkLR.scala index 2f41aeb376..52a0d69744 100644 --- a/examples/src/main/scala/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/spark/examples/SparkLR.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala index f598d2ff9c..00560ac9d1 100644 --- a/examples/src/main/scala/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/spark/examples/SparkPi.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import scala.math.random diff --git a/examples/src/main/scala/spark/examples/SparkTC.scala b/examples/src/main/scala/spark/examples/SparkTC.scala index 911ae8f168..bf988a953b 100644 --- a/examples/src/main/scala/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/spark/examples/SparkTC.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.examples import spark._ diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index 3b847fe603..f97174aeae 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import scala.collection.mutable.LinkedList diff --git a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala index 39c76fd98a..3ab4fc2c37 100644 --- a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.util.IntParam diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala index 9389f8a38d..f5baec242d 100644 --- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index 9202e65e09..4929703ba2 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import java.util.Properties diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala index 704540c2bf..150fb5eb9c 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala index f450e21040..da36c8c23c 100644 --- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.RDD diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala index 175281e095..7fb680bcc3 100644 --- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.util.IntParam diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index 51c3c9f9b4..33ab324732 100644 --- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming._ diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index 528778ed72..8770abd57e 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index 896e9fd8af..cba5c986be 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 65f0b6d352..682b99f75e 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala index 74d0d338b7..e264fae609 100644 --- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples import akka.actor.ActorSystem diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala index 4c6e08bc74..375d5c9d22 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples.clickstream import java.net.{InetAddress,ServerSocket,Socket,SocketException} diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala index e226a4a73a..a24342bebf 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.examples.clickstream import spark.streaming.{Seconds, StreamingContext} diff --git a/make-distribution.sh b/make-distribution.sh index feb13d52f9..0116215163 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -1,4 +1,22 @@ #!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # # Script to create a binary distribution for easy deploys of Spark. # The distribution directory defaults to dist/ but can be overridden below. @@ -12,6 +30,7 @@ # 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. # 4) ./bin/start-slave.sh 1 <> # 5) MASTER="spark://my-master-ip:7077" ./spark-shell +# # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala index b0e141ff32..d875d6de50 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.clustering import scala.collection.mutable.ArrayBuffer diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala index 4fd0646160..b8f80e80cd 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.clustering import spark.RDD diff --git a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala index e12b3be251..89fe7d7e85 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.clustering import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 90b0999a5e..2fb0c8136f 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index eff853f379..e1b73bc25e 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import spark.{Logging, RDD, SparkContext} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index ea80bfcbfd..b864fd4634 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 4c18cbdc6b..7da96397a6 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.recommendation import scala.collection.mutable.{ArrayBuffer, BitSet} diff --git a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala index fb812a6dbe..38637b3dd1 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.recommendation import spark.RDD diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index e4db7bb9b7..bb294c2257 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import spark.{Logging, RDD, SparkContext} diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala index 6e7c023bac..8094d22405 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala index f79974c191..645204ddf3 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Regression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Regression.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import spark.RDD diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 5f813df402..7c7f912b43 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import spark.{Logging, RDD, SparkContext} diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala index b83f505d8e..c2260ae286 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala index 08a031dded..b5e564df6d 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.util import spark.{RDD, SparkContext} diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 390c92763c..a112e0b506 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file core/target/unit-tests.log +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file core/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala index cb096f39a9..bebade9afb 100644 --- a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.clustering import scala.util.Random diff --git a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala index 2ada9ae76b..f98590b8d9 100644 --- a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.recommendation import scala.util.Random diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 04d3400cb4..bc9bfd054f 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala index df41dbbdff..3c588c6162 100644 --- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random diff --git a/pom.xml b/pom.xml index 6fa1f1d4f1..4b48072c6e 100644 --- a/pom.xml +++ b/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 org.spark-project diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 178a056de8..7a4d4c4575 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1,3 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ import sbt._ import sbt.Classpaths.publishTask diff --git a/project/build.properties b/project/build.properties index 9b860e23c5..08e17131f6 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1,18 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + sbt.version=0.12.3 diff --git a/pyspark b/pyspark index d662e90287..37a355462e 100755 --- a/pyspark +++ b/pyspark @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/python/epydoc.conf b/python/epydoc.conf index 45102cd9fe..d5d5aa5454 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -1,5 +1,22 @@ [epydoc] # Epydoc section marker (required by ConfigParser) +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Information about the project. name: PySpark url: http://spark-project.org diff --git a/python/examples/als.py b/python/examples/als.py index 010f80097f..f2b2eee64c 100755 --- a/python/examples/als.py +++ b/python/examples/als.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + """ This example requires numpy (http://www.numpy.org/) """ diff --git a/python/examples/kmeans.py b/python/examples/kmeans.py index 72cf9f88c6..c670556f2b 100644 --- a/python/examples/kmeans.py +++ b/python/examples/kmeans.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + """ This example requires numpy (http://www.numpy.org/) """ diff --git a/python/examples/logistic_regression.py b/python/examples/logistic_regression.py index f13698a86f..54d227d0d3 100755 --- a/python/examples/logistic_regression.py +++ b/python/examples/logistic_regression.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + """ This example requires numpy (http://www.numpy.org/) """ diff --git a/python/examples/pi.py b/python/examples/pi.py index 127cba029b..33c026e824 100644 --- a/python/examples/pi.py +++ b/python/examples/pi.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + import sys from random import random from operator import add diff --git a/python/examples/transitive_closure.py b/python/examples/transitive_closure.py index 7f85a1008e..40be3b5000 100644 --- a/python/examples/transitive_closure.py +++ b/python/examples/transitive_closure.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + import sys from random import Random diff --git a/python/examples/wordcount.py b/python/examples/wordcount.py index 857160624b..41c846ba79 100644 --- a/python/examples/wordcount.py +++ b/python/examples/wordcount.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + import sys from operator import add diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 3e9d7d36da..d367f91967 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + """ >>> from pyspark.context import SparkContext >>> sc = SparkContext('local', 'test') diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index def810dd46..dfdaba274f 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + """ >>> from pyspark.context import SparkContext >>> sc = SparkContext('local', 'test') diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 657fe6f989..2f741cb345 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + import os import shutil import sys diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 78c9457b84..eb18ec08c9 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + import os import signal import socket diff --git a/python/pyspark/files.py b/python/pyspark/files.py index 001b7a28b6..89bcbcfe06 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + import os diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 2329e536cc..e503fb7621 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + import os import sys from subprocess import Popen, PIPE diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index a9fec17a9d..c6a6b24c5a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + from base64 import standard_b64encode as b64enc import copy from collections import defaultdict diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 5a95144983..fecacd1241 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + import struct import cPickle diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 54ff1bf8e7..cc8cd9e3c4 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + """ An interactive shell. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 1e34d47365..dfd841b10a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + """ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 379bbfd4c2..75d692beeb 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + """ Worker that receives input from Piped RDD. """ diff --git a/python/run-tests b/python/run-tests index a3a9ff5dcb..1ee947d414 100755 --- a/python/run-tests +++ b/python/run-tests @@ -1,6 +1,24 @@ #!/usr/bin/env bash -# Figure out where the Scala framework is installed +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +# Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; cd ../; pwd)" FAILED=0 diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 7a7280313e..7c4e722cc1 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run index c54c9e97a0..0e5100e00b 100755 --- a/repl-bin/src/deb/bin/run +++ b/repl-bin/src/deb/bin/run @@ -1,6 +1,23 @@ #!/bin/bash -SCALA_VERSION=2.9.2 +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +SCALA_VERSION=2.9.3 # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor index 47b9cccdfe..d3c1c23413 100755 --- a/repl-bin/src/deb/bin/spark-executor +++ b/repl-bin/src/deb/bin/spark-executor @@ -1,5 +1,22 @@ #!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + FWDIR="$(cd `dirname $0`; pwd)" echo "Running spark-executor with framework dir = $FWDIR" exec $FWDIR/run spark.executor.MesosExecutorBackend diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell index 219c66eb0b..8b258543db 100755 --- a/repl-bin/src/deb/bin/spark-shell +++ b/repl-bin/src/deb/bin/spark-shell @@ -1,4 +1,21 @@ #!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + FWDIR="$(cd `dirname $0`; pwd)" exec $FWDIR/run spark.repl.Main "$@" diff --git a/repl/pom.xml b/repl/pom.xml index 92a2020b48..7d8da03254 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala index 0e9aa863b5..274bc585db 100644 --- a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.repl import java.io.{ByteArrayOutputStream, InputStream} diff --git a/repl/src/main/scala/spark/repl/Main.scala b/repl/src/main/scala/spark/repl/Main.scala index 58809ab646..d824d62fd1 100644 --- a/repl/src/main/scala/spark/repl/Main.scala +++ b/repl/src/main/scala/spark/repl/Main.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.repl import scala.collection.mutable.Set diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index cfb1a390e6..a6d33e69d2 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the repl/target/unit-tests.log +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the repl/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala index f46e6d8be4..80ae605558 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/spark/repl/ReplSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.repl import java.io._ diff --git a/run b/run index 6b5bc01a51..72ee76d722 100755 --- a/run +++ b/run @@ -1,5 +1,22 @@ #!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/run.cmd b/run.cmd index cc5605f8a9..c91764e617 100644 --- a/run.cmd +++ b/run.cmd @@ -1,2 +1,20 @@ @echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + cmd /V /E /C %~dp0run2.cmd %* diff --git a/run2.cmd b/run2.cmd index a9c4df180f..dc5e50931e 100644 --- a/run2.cmd +++ b/run2.cmd @@ -1,5 +1,22 @@ @echo off +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + set SCALA_VERSION=2.9.3 rem Figure out where the Spark framework is installed diff --git a/sbt/sbt b/sbt/sbt index 523fbb346b..8c9a42824f 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,8 +1,28 @@ #!/bin/bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + EXTRA_ARGS="" if [ "$MESOS_HOME" != "" ]; then EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java" fi + export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) export SPARK_TESTING=1 # To put test classes on classpath + java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd index ce3ae70174..56a16d77f2 100644 --- a/sbt/sbt.cmd +++ b/sbt/sbt.cmd @@ -1,5 +1,25 @@ @echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + set EXTRA_ARGS= if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\java + set SPARK_HOME=%~dp0.. + java -Xmx1200M -XX:MaxPermSize=200m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*" diff --git a/spark-executor b/spark-executor index b66c374ca8..feccbf5cc2 100755 --- a/spark-executor +++ b/spark-executor @@ -1,4 +1,22 @@ #!/bin/sh + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + FWDIR="`dirname $0`" echo "Running spark-executor with framework dir = $FWDIR" exec $FWDIR/run spark.executor.MesosExecutorBackend diff --git a/spark-shell b/spark-shell index a8e72143fb..5371fc540a 100755 --- a/spark-shell +++ b/spark-shell @@ -1,4 +1,22 @@ #!/bin/bash --posix + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # # Shell script for starting the Spark Shell REPL # Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} @@ -6,6 +24,7 @@ # Options: # -c Set the number of cores for REPL to use # + FWDIR="`dirname $0`" for o in "$@"; do diff --git a/spark-shell.cmd b/spark-shell.cmd index 34697d52d7..ec65eabb74 100644 --- a/spark-shell.cmd +++ b/spark-shell.cmd @@ -1,4 +1,22 @@ @echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + set FWDIR=%~dp0 set SPARK_LAUNCH_WITH_SCALA=1 cmd /V /E /C %FWDIR%run2.cmd spark.repl.Main %* diff --git a/streaming/pom.xml b/streaming/pom.xml index 2fb5bbdeb5..7e6b06d772 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 450e48d66e..1e4c1e3742 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.{Logging, Utils} diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index 9be7926a4a..684d3abb56 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.dstream._ diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala index 6b0fade7c6..399ca1c63d 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import org.apache.hadoop.fs.Path diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala index 3b331956f5..c09a332d44 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import dstream.InputDStream diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala index c2135195d8..12a14e233d 100644 --- a/streaming/src/main/scala/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/spark/streaming/Duration.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.Utils diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/spark/streaming/Interval.scala index 6a8b81760e..b30cd969e9 100644 --- a/streaming/src/main/scala/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/spark/streaming/Interval.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming private[streaming] diff --git a/streaming/src/main/scala/spark/streaming/Job.scala b/streaming/src/main/scala/spark/streaming/Job.scala index 67bd8388bc..ceb3f92b65 100644 --- a/streaming/src/main/scala/spark/streaming/Job.scala +++ b/streaming/src/main/scala/spark/streaming/Job.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import java.util.concurrent.atomic.AtomicLong diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala index 7696c4a592..a31230689f 100644 --- a/streaming/src/main/scala/spark/streaming/JobManager.scala +++ b/streaming/src/main/scala/spark/streaming/JobManager.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index b159d26c02..d4cf2e568c 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 8d0a83d439..47bf07bee1 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.StreamingContext._ diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala index 1c4b22a898..252cc2a303 100644 --- a/streaming/src/main/scala/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import util.{ManualClock, RecurringTimer, Clock} diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 36b841af8f..ffd656227d 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import akka.actor.Props diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/spark/streaming/Time.scala index f14decf08b..ad5eab9dd2 100644 --- a/streaming/src/main/scala/spark/streaming/Time.scala +++ b/streaming/src/main/scala/spark/streaming/Time.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming /** @@ -52,4 +69,4 @@ case class Time(private val millis: Long) { object Time { val ordering = Ordering.by((time: Time) => time.millis) -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala index 4d93f0a5f7..7dcb1d713d 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.api.java import spark.streaming.{Duration, Time, DStream} @@ -82,4 +99,4 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM object JavaDStream { implicit def fromDStream[T: ClassManifest](dstream: DStream[T]): JavaDStream[T] = new JavaDStream[T](dstream) -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index 548809a359..3ab5c1fdde 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.api.java import java.util.{List => JList} @@ -296,4 +313,4 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def checkpoint(interval: Duration) = { dstream.checkpoint(interval) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index 30240cad98..ccd15563b0 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.api.java import java.util.{List => JList} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index ed7b789d98..b7720ad0ea 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.api.java import spark.streaming._ diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala index 4ef4bb7de1..99553d295d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.{RDD, Partitioner} diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala index 41c3af4694..095137092a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD @@ -16,4 +33,4 @@ class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T override def compute(validTime: Time): Option[RDD[T]] = { Some(rdd) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala index 41b9bd9461..de0536125d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala index e993164f99..9d8c5c3175 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala index cabd34f5f2..78d7117f0f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala index a69af60589..d13bebb10f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala index c9644b3a83..4906f503c2 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.StreamingContext diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala index ee69ea5177..7df537eb56 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala index b589cbd4d5..06fda6fe8e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala index 3c5d43a609..4dbdec459d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Time, Duration, StreamingContext, DStream} diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 55d2957be4..6ee588af15 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala index 848afecfad..af41a1b9ac 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala index 6055aa6a05..8d8a6161c6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala index 20818a0cab..3fda84a38a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 122a529bb7..344b41c4d0 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver} diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala index 3c2a81947b..33f7cd063f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.StreamingContext diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala index da224ad6f7..b269061b73 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala index 1b2fa56779..236f74f575 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala index 343b6915e7..96260501ab 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.StreamingContext._ diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala index 1f9548bfb8..83b57b27f7 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.{RDD, Partitioner} diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala index 1408af0afa..5877b10e0e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.StreamingContext diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala index db62955036..4b46613d5e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala index 99660d9dee..e7fbc5bbcf 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index ff7a58be45..f09a8b9f90 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark._ diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala index 00bad5da34..3eaa9a7e7f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala index cbf0c88108..fd24d61730 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index 036c95a860..2d9937eab8 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.receivers import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala index 5533c3cf1e..22d554e7e4 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.receivers import akka.actor.Actor diff --git a/streaming/src/main/scala/spark/streaming/util/Clock.scala b/streaming/src/main/scala/spark/streaming/util/Clock.scala index 974651f9f6..d9ac722df5 100644 --- a/streaming/src/main/scala/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/spark/streaming/util/Clock.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util private[streaming] diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index 426a9b6f71..8ce5d8daf5 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util import spark.{Logging, RDD} diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala index 03749d4a94..bf04120293 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util import spark.SparkContext diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala index bd0b0e74c1..5cc6ad9dee 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util import java.nio.ByteBuffer diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala index 8e10276deb..7ecc44236d 100644 --- a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming.util private[streaming] diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 4cf10582a9..3b93790baa 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming; import com.google.common.base.Optional; diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala index 64a7e7cbf9..f9d25db8da 100644 --- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import collection.mutable.{SynchronizedBuffer, ArrayBuffer} @@ -64,4 +81,4 @@ object JavaTestUtils extends JavaTestBase { object JavaCheckpointTestUtils extends JavaTestBase { override def actuallyWait = true -} \ No newline at end of file +} diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 59c445e63f..063529a9cb 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file streaming/target/unit-tests.log +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index 565089a853..67e3e0cd30 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.StreamingContext._ diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index 607dea77ec..8c639648f0 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import dstream.FileInputDStream diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala index a5fa7ab92d..7fc649fe27 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.Logging diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index b024fc9dcc..1c5419b16d 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import akka.actor.Actor diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala index ad6aa79d10..cb34b5a7cc 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.dstream.{InputDStream, ForEachDStream} diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala index 80d827706f..894b765fc6 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.streaming import spark.streaming.StreamingContext._ From 4ff494de20c36151dc29a60825d67e094d14acd4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 17:26:48 -0700 Subject: [PATCH 217/419] Some missing license headers --- bin/compute-classpath.cmd | 17 +++++++++++++++++ bin/compute-classpath.sh | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 44826f339c..835d1af794 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -1,5 +1,22 @@ @echo off +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" rem script and the ExecutorRunner in standalone cluster mode. diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index bd48b43da3..4c539649a2 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -1,5 +1,22 @@ #!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + # This script computes Spark's classpath and prints it to stdout; it's used by both the "run" # script and the ExecutorRunner in standalone cluster mode. From a1d2c3434657f0d901a2f8cfddf62065b5ad4385 Mon Sep 17 00:00:00 2001 From: ctn Date: Wed, 17 Jul 2013 11:27:11 -0700 Subject: [PATCH 218/419] [BUGFIX] Fix for sbt/sbt script SPARK_HOME setting In some environments, this command export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) echoes two paths, one by the "cd ..", and one by the "pwd". Note the resulting erroneous -jar paths below: ctn@ubuntu:~/src/spark$ sbt/sbt + EXTRA_ARGS= + '[' '' '!=' '' ']' +++ dirname sbt/sbt ++ cd sbt/.. ++ pwd + export 'SPARK_HOME=/home/ctn/src/spark /home/ctn/src/spark' + SPARK_HOME='/home/ctn/src/spark /home/ctn/src/spark' + export SPARK_TESTING=1 + SPARK_TESTING=1 + java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m -jar /home/ctn/src/spark /home/ctn/src/spark/sbt/sbt-launch-0.11.3-2.jar Error: Invalid or corrupt jarfile /home/ctn/src/spark Committer: ctn On branch master Changes to be committed: - Send output of the "cd .." part to /dev/null modified: sbt/sbt --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 8c9a42824f..d64645e32d 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -22,7 +22,7 @@ if [ "$MESOS_HOME" != "" ]; then EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java" fi -export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) +export SPARK_HOME=$(cd "$(dirname $0)/.." 2>&1 >/dev/null ; pwd) export SPARK_TESTING=1 # To put test classes on classpath java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" From 84fa20c2a135f54745ddde9abb4f5e60af8856d1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:04:05 -0700 Subject: [PATCH 219/419] Allow initial weight vectors in LogisticRegression. Also move LogisticGradient to the LogisticRegression file and fix the unit tests log path. --- .../spark/mllib/optimization/Gradient.scala | 18 ------ .../mllib/optimization/GradientDescent.scala | 5 +- .../mllib/regression/LogisticRegression.scala | 61 ++++++++++++++++++- mllib/src/test/resources/log4j.properties | 2 +- .../regression/LogisticRegressionSuite.scala | 52 +++++++++++++--- 5 files changed, 106 insertions(+), 32 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 2fb0c8136f..d5338360c8 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -30,21 +30,3 @@ abstract class Gradient extends Serializable { def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): (DoubleMatrix, Double) } - -class LogisticGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { - val margin: Double = -1.0 * data.dot(weights) - val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - - val gradient = data.mul(gradientMultiplier) - val loss = - if (margin > 0) { - math.log(1 + math.exp(0 - margin)) - } else { - math.log(1 + math.exp(margin)) - margin - } - - (gradient, loss) - } -} diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index e1b73bc25e..2ac0808357 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -48,16 +48,17 @@ object GradientDescent { updater: Updater, stepSize: Double, numIters: Int, + initialWeights: Array[Double], miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { val lossHistory = new ArrayBuffer[Double](numIters) - val nfeatures: Int = data.take(1)(0)._2.length val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction // Initialize weights as a column matrix - var weights = DoubleMatrix.ones(nfeatures) + var weights = new DoubleMatrix(1, initialWeights.length, + initialWeights:_*) var reg_val = 0.0 for (i <- 1 to numIters) { diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index bb294c2257..dab15aa386 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -46,6 +46,24 @@ class LogisticRegressionModel( } } +class LogisticGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + val margin: Double = -1.0 * data.dot(weights) + val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label + + val gradient = data.mul(gradientMultiplier) + val loss = + if (margin > 0) { + math.log(1 + math.exp(0 - margin)) + } else { + math.log(1 + math.exp(margin)) - margin + } + + (gradient, loss) + } +} + class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -80,13 +98,30 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val initialWeights = Array.fill(nfeatures)(1.0) + train(input, initialWeights) + } + + def train( + input: RDD[(Double, Array[Double])], + initialWeights: Array[Double]): LogisticRegressionModel = { + // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => (y, Array(1.0, features:_*)) } + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) + val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + data, + new LogisticGradient(), + new SimpleUpdater(), + stepSize, + numIters, + initalWeightsWithIntercept, + miniBatchFraction) val weightsScaled = weights.getRange(1, weights.length) val intercept = weights.get(0) @@ -105,6 +140,30 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D */ object LogisticRegression { + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LogisticRegressionModel = + { + new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index a112e0b506..4265ba6e5d 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=ml/target/unit-tests.log +log4j.appender.file.file=mllib/target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index bc9bfd054f..2ff248d256 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -34,16 +34,14 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { - val nPoints = 10000 + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int) : Seq[(Double, Array[Double])] = { val rnd = new Random(42) - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val A = 2.0 - val B = -1.5 - // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) val unifRand = new scala.util.Random(45) val rLogis = (0 until nPoints).map { i => @@ -51,14 +49,24 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { math.log(u) - math.log(1.0-u) } - // y <- A + B*x + rlogis(100) + // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) val y = (0 until nPoints).map { i => - val yVal = A + B * x1(i) + rLogis(i) + val yVal = offset + scale * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))) + testData + } + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints) val testRDD = sc.parallelize(testData, 2) testRDD.cache() @@ -71,4 +79,28 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") } + + test("logistic regression with initial weights") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints) + + val initialB = -1.0 + val initialWeights = Array(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new LogisticRegression().setStepSize(10.0) + .setNumIterations(10) + + val model = lr.train(testRDD, initialWeights) + + val weight0 = model.weights.get(0) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + } } From 64b88e039a66fffdc1d0f0eb1caa9328d76bbb06 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:11:44 -0700 Subject: [PATCH 220/419] Move ML lib data generator files to util/ --- .../{regression => util}/LogisticRegressionGenerator.scala | 2 +- .../mllib/{regression => util}/RidgeRegressionGenerator.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename mllib/src/main/scala/spark/mllib/{regression => util}/LogisticRegressionGenerator.scala (98%) rename mllib/src/main/scala/spark/mllib/{regression => util}/RidgeRegressionGenerator.scala (98%) diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala index 8094d22405..4c580b44da 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression +package spark.mllib.util import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala index c2260ae286..a5bb2e762f 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression +package spark.mllib.util import scala.util.Random From 3bf989713654129ad35a80309d1b354ca5ddd06c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:20:24 -0700 Subject: [PATCH 221/419] Rename loss -> stochasticLoss and add a note to explain why we have multiple train methods. --- .../spark/mllib/optimization/GradientDescent.scala | 9 +++++---- .../spark/mllib/regression/LogisticRegression.scala | 10 ++++++---- .../scala/spark/mllib/regression/RidgeRegression.scala | 2 ++ 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 2ac0808357..77f5a7ae24 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -40,7 +40,8 @@ object GradientDescent { * one iteration of SGD. Default value 1.0. * * @return weights - Column matrix containing weights for every feature. - * @return lossHistory - Array containing the loss computed for every iteration. + * @return stochasticLossHistory - Array containing the stochastic loss computed for + * every iteration. */ def runMiniBatchSGD( data: RDD[(Double, Array[Double])], @@ -51,7 +52,7 @@ object GradientDescent { initialWeights: Array[Double], miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { - val lossHistory = new ArrayBuffer[Double](numIters) + val stochasticLossHistory = new ArrayBuffer[Double](numIters) val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction @@ -69,12 +70,12 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - lossHistory.append(lossSum / miniBatchSize + reg_val) + stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) weights = update._1 reg_val = update._2 } - (weights, lossHistory.toArray) + (weights, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index dab15aa386..664baf33a3 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -30,7 +30,7 @@ import org.jblas.DoubleMatrix class LogisticRegressionModel( val weights: DoubleMatrix, val intercept: Double, - val losses: Array[Double]) extends RegressionModel { + val stochasticLosses: Array[Double]) extends RegressionModel { override def predict(testData: spark.RDD[Array[Double]]) = { testData.map { x => @@ -114,7 +114,7 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val (weights, losses) = GradientDescent.runMiniBatchSGD( + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( data, new LogisticGradient(), new SimpleUpdater(), @@ -126,17 +126,19 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D val weightsScaled = weights.getRange(1, weights.length) val intercept = weights.get(0) - val model = new LogisticRegressionModel(weightsScaled, intercept, losses) + val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) logInfo("Final model weights " + model.weights) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochastic losses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling Logistic Regression. + * NOTE(shivaram): We use multiple train methods instead of default arguments to support + * Java programs. */ object LogisticRegression { diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 7c7f912b43..f724edd732 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -164,6 +164,8 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) /** * Top-level methods for calling Ridge Regression. + * NOTE(shivaram): We use multiple train methods instead of default arguments to support + * Java programs. */ object RidgeRegression { From 45f3c855181539306d5610c5aa265f24b431c142 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 16:03:29 -0700 Subject: [PATCH 222/419] Change weights to be Array[Double] in LR model. Also ensure weights are initialized to a column vector. --- .../mllib/optimization/GradientDescent.scala | 5 ++--- .../mllib/regression/LogisticRegression.scala | 17 +++++++++++------ .../regression/LogisticRegressionSuite.scala | 4 ++-- 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 77f5a7ae24..2c5038757b 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -57,9 +57,8 @@ object GradientDescent { val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction - // Initialize weights as a column matrix - var weights = new DoubleMatrix(1, initialWeights.length, - initialWeights:_*) + // Initialize weights as a column vector + var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) var reg_val = 0.0 for (i <- 1 to numIters) { diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index 664baf33a3..ab865af0c6 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -28,20 +28,23 @@ import org.jblas.DoubleMatrix * Based on Matlab code written by John Duchi. */ class LogisticRegressionModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, val stochasticLosses: Array[Double]) extends RegressionModel { + // Create a column vector that can be used for predictions + private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) + override def predict(testData: spark.RDD[Array[Double]]) = { testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(weightsMatrix).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - val margin = dataMat.mmul(this.weights).get(0) + this.intercept + val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } @@ -123,12 +126,14 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D initalWeightsWithIntercept, miniBatchFraction) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val weightsArray = weights.toArray() + + val intercept = weightsArray(0) + val weightsScaled = weightsArray.tail val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) - logInfo("Final model weights " + model.weights) + logInfo("Final model weights " + model.weights.mkString(",")) logInfo("Final model intercept " + model.intercept) logInfo("Last 10 stochastic losses " + model.stochasticLosses.takeRight(10).mkString(", ")) model diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 2ff248d256..47191d9a5a 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -75,7 +75,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val model = lr.train(testRDD) - val weight0 = model.weights.get(0) + val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") } @@ -99,7 +99,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val model = lr.train(testRDD, initialWeights) - val weight0 = model.weights.get(0) + val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") } From 217667174e267adba5469cf26b3e4418e3d1cc90 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 16:08:34 -0700 Subject: [PATCH 223/419] Return Array[Double] from SGD instead of DoubleMatrix --- .../scala/spark/mllib/optimization/GradientDescent.scala | 4 ++-- .../scala/spark/mllib/regression/LogisticRegression.scala | 6 ++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 2c5038757b..4c996c0903 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -50,7 +50,7 @@ object GradientDescent { stepSize: Double, numIters: Int, initialWeights: Array[Double], - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + miniBatchFraction: Double=1.0) : (Array[Double], Array[Double]) = { val stochasticLossHistory = new ArrayBuffer[Double](numIters) @@ -75,6 +75,6 @@ object GradientDescent { reg_val = update._2 } - (weights, stochasticLossHistory.toArray) + (weights.toArray, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index ab865af0c6..711e205c39 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -126,10 +126,8 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D initalWeightsWithIntercept, miniBatchFraction) - val weightsArray = weights.toArray() - - val intercept = weightsArray(0) - val weightsScaled = weightsArray.tail + val intercept = weights(0) + val weightsScaled = weights.tail val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) From 88a0823c58304fa1b290667d95a482105a8a5f52 Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Thu, 18 Jul 2013 00:51:18 +0000 Subject: [PATCH 224/419] Consistently invoke bash with /usr/bin/env bash in scripts to make code more portable (JIRA Ticket SPARK-817) --- bin/compute-classpath.sh | 2 +- docs/spark-simple-tutorial.md | 2 +- ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh | 2 +- ec2/deploy.generic/root/spark-ec2/ec2-variables.sh | 2 +- repl-bin/src/deb/bin/run | 2 +- repl-bin/src/deb/bin/spark-executor | 2 +- repl-bin/src/deb/bin/spark-shell | 2 +- run | 2 +- sbt/sbt | 2 +- spark-shell | 5 ++++- 10 files changed, 13 insertions(+), 10 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 4c539649a2..e6fdc5306e 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/docs/spark-simple-tutorial.md b/docs/spark-simple-tutorial.md index 9875de62bd..fbdbc7d19d 100644 --- a/docs/spark-simple-tutorial.md +++ b/docs/spark-simple-tutorial.md @@ -13,7 +13,7 @@ title: Tutorial - Running a Simple Spark Application 3. Edit the ~/SparkTest/sbt/sbt file to look like this: - #!/bin/bash + #!/usr/bin/env bash java -Xmx800M -XX:MaxPermSize=150m -jar $(dirname $0)/sbt-launch-*.jar "$@" 4. To build a Spark application, you need Spark and its dependencies in a single Java archive (JAR) file. Create this JAR in Spark's main directory with sbt as: diff --git a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh b/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh index 50ecf83404..ede6c78428 100644 --- a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # These variables are automatically filled in by the mesos-ec2 script. export MESOS_MASTERS="{{master_list}}" diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 166a884c88..685ed8be8c 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # These variables are automatically filled in by the mesos-ec2 script. export MESOS_MASTERS="{{master_list}}" diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run index 0e5100e00b..8b5d8300f2 100755 --- a/repl-bin/src/deb/bin/run +++ b/repl-bin/src/deb/bin/run @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor index d3c1c23413..bcfae22677 100755 --- a/repl-bin/src/deb/bin/spark-executor +++ b/repl-bin/src/deb/bin/spark-executor @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell index 8b258543db..ec7e33e1e3 100755 --- a/repl-bin/src/deb/bin/spark-shell +++ b/repl-bin/src/deb/bin/spark-shell @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/run b/run index 72ee76d722..f49aa92796 100755 --- a/run +++ b/run @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/sbt/sbt b/sbt/sbt index 8c9a42824f..b6f91f5085 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/spark-shell b/spark-shell index 5371fc540a..31a4138124 100755 --- a/spark-shell +++ b/spark-shell @@ -1,4 +1,4 @@ -#!/bin/bash --posix +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -25,6 +25,9 @@ # -c Set the number of cores for REPL to use # +# Enter posix mode for bash +set -o posix + FWDIR="`dirname $0`" for o in "$@"; do From 3aad45265324537e51cab79db3f09d9310339f77 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 02:29:46 +0800 Subject: [PATCH 225/419] fix a bug in build process that pulls in two versionf of ASM. --- project/SparkBuild.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e9819b877d..59edda5097 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -183,10 +183,10 @@ object SparkBuild extends Build { if (HADOOP_YARN) { Seq( // Exclude rule required for all ? - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm) ) } else { Seq( From 7ab1170503ab46ffd838fb98590dd3d66748dd38 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 18 Jul 2013 11:55:19 -0700 Subject: [PATCH 226/419] Refactor data generators to have a function that can be used in tests. --- .../util/LogisticRegressionGenerator.scala | 48 ++++++++----- .../mllib/util/RidgeRegressionGenerator.scala | 67 ++++++++++++------- 2 files changed, 76 insertions(+), 39 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala index 4c580b44da..8d659cd97c 100644 --- a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala @@ -19,12 +19,39 @@ package spark.mllib.util import scala.util.Random -import org.jblas.DoubleMatrix - import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils -object LogisticRegressionGenerator { +object LogisticRegressionDataGenerator { + + /** + * Generate an RDD containing test data for LogisticRegression. This function chooses + * positive labels with probability `probOne` and scales positive examples by `eps`. + * + * @param sc SparkContext to use for creating the RDD. + * @param nexamples Number of examples that will be contained in the RDD. + * @param nfeatures Number of features to generate for each example. + * @param eps Epsilon factor by which positive examples are scaled. + * @param nparts Number of partitions of the generated RDD. Default value is 2. + * @param probOne Probability that a label is 1 (and not 0). Default value is 0.5. + */ + def generateLogisticRDD( + sc: SparkContext, + nexamples: Int, + nfeatures: Int, + eps: Double, + nparts: Int = 2, + probOne: Double = 0.5): RDD[(Double, Array[Double])] = { + val data = sc.parallelize(0 until nexamples, nparts).map { idx => + val rnd = new Random(42 + idx) + + val y = if (idx % 2 == 0) 0.0 else 1.0 + val x = Array.fill[Double](nfeatures) { + rnd.nextGaussian() + (y * eps) + } + (y, x) + } + data + } def main(args: Array[String]) { if (args.length != 5) { @@ -40,17 +67,8 @@ object LogisticRegressionGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 3 - val sc = new SparkContext(sparkMaster, "LogisticRegressionGenerator") - - val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => - val rnd = new Random(42 + idx) - - val y = if (idx % 2 == 0) 0 else 1 - val x = Array.fill[Double](nfeatures) { - rnd.nextGaussian() + (y * eps) - } - (y, x) - } + val sc = new SparkContext(sparkMaster, "LogisticRegressionDataGenerator") + val data = generateLogisticRDD(sc, nexamples, nfeatures, eps, parts) MLUtils.saveLabeledData(data, outputPath) sc.stop() diff --git a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala index a5bb2e762f..6861913dc7 100644 --- a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala @@ -22,11 +22,52 @@ import scala.util.Random import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils object RidgeRegressionGenerator { + /** + * Generate an RDD containing test data used for RidgeRegression. This function generates + * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the + * response variable `Y`. + * + * @param sc SparkContext to be used for generating the RDD. + * @param nexamples Number of examples that will be contained in the RDD. + * @param nfeatures Number of features to generate for each example. + * @param eps Epsilon factor by which examples are scaled. + * @param nparts Number of partitions in the RDD. Default value is 2. + */ + def generateRidgeRDD( + sc: SparkContext, + nexamples: Int, + nfeatures: Int, + eps: Double, + nparts: Int = 2) : RDD[(Double, Array[Double])] = { + org.jblas.util.Random.seed(42) + // Random values distributed uniformly in [-0.5, 0.5] + val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) + w.put(0, 0, 10) + w.put(1, 0, 10) + + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nparts, nparts).flatMap { p => + org.jblas.util.Random.seed(42 + p) + val examplesInPartition = nexamples / nparts + + val X = DoubleMatrix.rand(examplesInPartition, nfeatures) + val y = X.mmul(w) + + val rnd = new Random(42 + p) + + val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) + val yObs = new DoubleMatrix(normalValues).addi(y) + + Iterator.tabulate(examplesInPartition) { i => + (yObs.get(i, 0), X.getRow(i).toArray) + } + } + data + } + def main(args: Array[String]) { if (args.length != 5) { println("Usage: RidgeRegressionGenerator " + @@ -41,30 +82,8 @@ object RidgeRegressionGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 10 - org.jblas.util.Random.seed(42) val sc = new SparkContext(sparkMaster, "RidgeRegressionGenerator") - - // Random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - w.put(0, 0, 10) - w.put(1, 0, 10) - - val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until parts, parts).flatMap { p => - org.jblas.util.Random.seed(42 + p) - val examplesInPartition = nexamples / parts - - val X = DoubleMatrix.rand(examplesInPartition, nfeatures) - val y = X.mmul(w) - - val rnd = new Random(42 + p) - - val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) - val yObs = new DoubleMatrix(normalValues).addi(y) - - Iterator.tabulate(examplesInPartition) { i => - (yObs.get(i, 0), X.getRow(i).toArray) - } - } + val data = generateRidgeRDD(sc, nexamples, nfeatures, eps, parts) MLUtils.saveLabeledData(data, outputPath) sc.stop() From 2c9ea56db4a1f4324f1dc8fadaf2491d83eda8ca Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 18 Jul 2013 11:57:14 -0700 Subject: [PATCH 227/419] Rename classes to be called DataGenerator --- ...Generator.scala => LogisticRegressionDataGenerator.scala} | 0 ...ionGenerator.scala => RidgeRegressionDataGenerator.scala} | 5 ++--- 2 files changed, 2 insertions(+), 3 deletions(-) rename mllib/src/main/scala/spark/mllib/util/{LogisticRegressionGenerator.scala => LogisticRegressionDataGenerator.scala} (100%) rename mllib/src/main/scala/spark/mllib/util/{RidgeRegressionGenerator.scala => RidgeRegressionDataGenerator.scala} (96%) diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala index 6861913dc7..c5b8a29942 100644 --- a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala @@ -23,8 +23,7 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} - -object RidgeRegressionGenerator { +object RidgeRegressionDataGenerator { /** * Generate an RDD containing test data used for RidgeRegression. This function generates @@ -82,7 +81,7 @@ object RidgeRegressionGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 10 - val sc = new SparkContext(sparkMaster, "RidgeRegressionGenerator") + val sc = new SparkContext(sparkMaster, "RidgeRegressionDataGenerator") val data = generateRidgeRDD(sc, nexamples, nfeatures, eps, parts) MLUtils.saveLabeledData(data, outputPath) From a613628c5078cf41feb973d0ee8a06eb69615bcf Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 16:59:12 +0800 Subject: [PATCH 228/419] Do not copy local jars given to SparkContext in yarn mode since the Context is not running on local. This bug causes failure when jars can not be found. Example codes (such as spark.examples.SparkPi) can not work without this fix under yarn mode. --- core/src/main/scala/spark/SparkContext.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 46b9935cb7..957c541ecf 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -102,6 +102,7 @@ class SparkContext( } private val isLocal = (master == "local" || master.startsWith("local[")) + private val isYarn = (master == "yarn-standalone") // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( @@ -577,11 +578,18 @@ class SparkContext( } else { val uri = new URI(path) val key = uri.getScheme match { - case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) + case null | "file" => + if (!isYarn) + env.httpFileServer.addJar(new File(uri.getPath)) + else + null case _ => path } - addedJars(key) = System.currentTimeMillis - logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) + + if (key != null) { + addedJars(key) = System.currentTimeMillis + logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) + } } } From aa6f83289b87f38481dbae60ad91d2ac78ccea46 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 22:25:28 +0800 Subject: [PATCH 229/419] A better fix for giving local jars unde Yarn mode. --- .../scala/spark/deploy/SparkHadoopUtil.scala | 3 +++ .../scala/spark/deploy/SparkHadoopUtil.scala | 3 +++ core/src/main/scala/spark/SparkContext.scala | 17 +++++++---------- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index df55be1254..9f040faac3 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -41,4 +41,7 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} + + def isYarnMode(): Boolean = { False } + } diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index df55be1254..9f040faac3 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -41,4 +41,7 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} + + def isYarnMode(): Boolean = { False } + } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 957c541ecf..c01e315e35 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -102,7 +102,6 @@ class SparkContext( } private val isLocal = (master == "local" || master.startsWith("local[")) - private val isYarn = (master == "yarn-standalone") // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( @@ -579,17 +578,15 @@ class SparkContext( val uri = new URI(path) val key = uri.getScheme match { case null | "file" => - if (!isYarn) - env.httpFileServer.addJar(new File(uri.getPath)) - else - null + if (SparkHadoopUtil.isYarnMode()) { + logWarning("local jar specified as parameter to addJar under Yarn mode") + return + } + env.httpFileServer.addJar(new File(uri.getPath)) case _ => path } - - if (key != null) { - addedJars(key) = System.currentTimeMillis - logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) - } + addedJars(key) = System.currentTimeMillis + logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } } From 4530e8a9bfe35b6d562876b1fb66e534ff5c286d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Jul 2013 00:04:25 +0800 Subject: [PATCH 230/419] fix typo. --- core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala | 2 +- core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index 9f040faac3..617954cb98 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -42,6 +42,6 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} - def isYarnMode(): Boolean = { False } + def isYarnMode(): Boolean = { false } } diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index 9f040faac3..617954cb98 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -42,6 +42,6 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} - def isYarnMode(): Boolean = { False } + def isYarnMode(): Boolean = { false } } From d1738d72ba2ff28c38e03beb3e17f03d3dd77e1a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Jul 2013 00:37:24 +0800 Subject: [PATCH 231/419] also exclude asm for hadoop2. hadoop1 looks like no need to do that too. --- project/SparkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 59edda5097..2f2cbf646a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -190,8 +190,8 @@ object SparkBuild extends Build { ) } else { Seq( - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm) ) } } else { From cfce9a6a365a8a4b156e99f412157704cab592b9 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Thu, 18 Jul 2013 17:43:27 -0700 Subject: [PATCH 232/419] Regression: default webui-port can't be set via command line "--webui-port" anymore --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- .../main/scala/spark/deploy/master/MasterArguments.scala | 5 ++++- .../src/main/scala/spark/deploy/master/ui/MasterWebUI.scala | 6 ++---- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index e5a7a87e2e..eddcafd84d 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -53,7 +53,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act var firstApp: Option[ApplicationInfo] = None - val webUi = new MasterWebUI(self) + val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala index d0ec3d5ea0..0ae0160767 100644 --- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala @@ -38,7 +38,10 @@ private[spark] class MasterArguments(args: Array[String]) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } - + if (System.getProperty("master.ui.port") != null) { + webUiPort = System.getProperty("master.ui.port").toInt + } + parse(args.toList) def parse(args: List[String]): Unit = args match { diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index 04b32c7968..dabc2d8dc7 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -32,12 +32,11 @@ import spark.ui.JettyUtils._ * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) extends Logging { +class MasterWebUI(val master: ActorRef, requestedPort: Int) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() - val port = requestedPort.getOrElse( - System.getProperty("master.ui.port", MasterWebUI.DEFAULT_PORT).toInt) + val port = requestedPort var server: Option[Server] = None var boundPort: Option[Int] = None @@ -72,5 +71,4 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten private[spark] object MasterWebUI { val STATIC_RESOURCE_DIR = "spark/ui/static" - val DEFAULT_PORT = "8080" } From 81bb5dc64007121df52cbafdf5f025c97f145953 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 19 Jul 2013 14:08:30 -0700 Subject: [PATCH 233/419] Creates Executors tab for application with RDD block and memory/disk used, solves SPARK-808 --- .../main/scala/spark/executor/Executor.scala | 1 + .../scala/spark/executor/TaskMetrics.scala | 5 ++ core/src/main/scala/spark/ui/Page.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 5 +- core/src/main/scala/spark/ui/UIUtils.scala | 5 ++ .../scala/spark/ui/exec/ExecutorsUI.scala | 79 +++++++++++++++++++ .../scala/spark/ui/storage/IndexPage.scala | 18 +---- 7 files changed, 96 insertions(+), 19 deletions(-) create mode 100644 core/src/main/scala/spark/ui/exec/ExecutorsUI.scala diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 2e81151882..82e5f5a015 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -126,6 +126,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => m.hostname = Utils.localHostName + m.executorId = executorId m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 3151627839..e14dc57501 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -23,6 +23,11 @@ class TaskMetrics extends Serializable { */ var hostname: String = _ + /** + * Executor's ID the task runs on + */ + var executorId: String = _ + /** * Time taken on the executor to deserialize this task */ diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala index a31e750d06..03034a4520 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/spark/ui/Page.scala @@ -17,4 +17,4 @@ package spark.ui -private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } +private[spark] object Page extends Enumeration { val Storage, Jobs, Environment, Executors = Value } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 9396f22063..1534705bc5 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -23,6 +23,7 @@ import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, SparkContext, Utils} import spark.ui.env.EnvironmentUI +import spark.ui.exec.ExecutorsUI import spark.ui.storage.BlockManagerUI import spark.ui.jobs.JobProgressUI import spark.ui.JettyUtils._ @@ -41,7 +42,9 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) val env = new EnvironmentUI(sc) - val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ handlers + val exec = new ExecutorsUI(sc) + val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ + exec.getHandlers ++ handlers /** Bind the HTTP server which backs this web interface */ def bind() { diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index b1d11954dd..e33c80282a 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -40,6 +40,10 @@ private[spark] object UIUtils { case Environment =>
      • Environment
      • case _ =>
      • Environment
      • } + val executors = page match { + case Executors =>
      • Executors
      • + case _ =>
      • Executors
      • + } @@ -66,6 +70,7 @@ private[spark] object UIUtils { {storage} {jobs} {environment} + {executors}
      • Application: {sc.appName}
      • diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala new file mode 100644 index 0000000000..de0e571553 --- /dev/null +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -0,0 +1,79 @@ +package spark.ui.exec + + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.util.Properties + +import spark.scheduler.cluster.TaskInfo +import spark.executor.TaskMetrics +import spark.ui.JettyUtils._ +import spark.ui.UIUtils.headerSparkPage +import spark.ui.Page.Executors +import spark.storage.{StorageStatus, StorageUtils} +import spark.SparkContext +import spark.ui.UIUtils +import spark.Utils + +import scala.xml.{Node, XML} + +private[spark] class ExecutorsUI(val sc: SparkContext) { + + def getHandlers = Seq[(String, Handler)]( + ("/executors", (request: HttpServletRequest) => render(request)) + ) + + def render(request: HttpServletRequest): Seq[Node] = { + val storageStatusList = sc.getExecutorStorageStatus + + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + + val execTables = + for (a <- 0 until storageStatusList.size) + yield getExecTable(a) + + val content = +
        +
        +
          +
        • Memory: + {Utils.memoryBytesToString(maxMem - remainingMem)} Used + ({Utils.memoryBytesToString(remainingMem)} Available)
        • +
        • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
        • +
        +
        +
        +
        +
        + {execTables} +
        +
        ; + + headerSparkPage(content, sc, "Executors", Executors) + } + + def getExecTable(a: Int): Seq[Node] = { + val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) + val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) + val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) + val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString + val execInfo = Seq( + ("RDD blocks", rddBlocks), + ("Memory used", "%s/%s".format(memUsed, maxMem)), + ("Disk used", diskUsed) + ) + def execRow(kv: (String, String)) = {kv._1}{kv._2} + val table = UIUtils.listingTable(Seq("Name", "Value"), execRow, execInfo) + val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId + val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort + val header = +

        Executor {execId}

        +

        {hostPort}

        ; + header ++ table + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 4e0360d19a..f76192eba8 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -33,10 +33,6 @@ private[spark] class IndexPage(parent: BlockManagerUI) { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) val rddHeaders = Seq( "RDD Name", @@ -46,19 +42,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) { "Size in Memory", "Size on Disk") val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val rddTable = listingTable(rddHeaders, rddRow, rdds) - - val content = -
        -
        -
          -
        • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
        • -
        • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
        • -
        -
        -
        ++ {rddTable}; + val content = listingTable(rddHeaders, rddRow, rdds) headerSparkPage(content, parent.sc, "Spark Storage ", Storage) } From 865dc63bac1cb72ecb84038c0a7cd708cc9e19d7 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 19 Jul 2013 15:57:01 -0700 Subject: [PATCH 234/419] Changed table format for executors --- .../scala/spark/ui/exec/ExecutorsUI.scala | 40 +++++++++++-------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index de0e571553..fa516a4a34 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -33,9 +33,19 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - val execTables = - for (a <- 0 until storageStatusList.size) - yield getExecTable(a) + val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used") + def execRow(kv: Seq[String]) = + + {kv(0)} + {kv(1)} + {kv(2)} + {kv(3)} + {kv(4)} + + val execInfo = + for (b <- 0 until storageStatusList.size) + yield getExecInfo(b) + val execTable = UIUtils.listingTable(execHead, execRow, execInfo) val content =
        @@ -50,30 +60,26 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
        - {execTables} + {execTable}
        ; headerSparkPage(content, sc, "Executors", Executors) } - def getExecTable(a: Int): Seq[Node] = { + def getExecInfo(a: Int): Seq[String] = { + val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId + val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString - val execInfo = Seq( - ("RDD blocks", rddBlocks), - ("Memory used", "%s/%s".format(memUsed, maxMem)), - ("Disk used", diskUsed) + Seq( + execId, + hostPort, + rddBlocks, + "%s/%s".format(memUsed, maxMem), + diskUsed ) - def execRow(kv: (String, String)) = {kv._1}{kv._2} - val table = UIUtils.listingTable(Seq("Name", "Value"), execRow, execInfo) - val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId - val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val header = -

        Executor {execId}

        -

        {hostPort}

        ; - header ++ table } } \ No newline at end of file From f4d514810e6fd9f42868ebb9a89390c62c3b42e1 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Tue, 2 Jul 2013 17:03:24 -0700 Subject: [PATCH 235/419] Building spark assembly for further consumption of the Spark project with a deployed cluster --- assembly/README | 13 +++ assembly/pom.xml | 104 ++++++++++++++++++++++++ assembly/src/main/assembly/assembly.xml | 68 ++++++++++++++++ pom.xml | 21 ++++- spark-shell | 1 + 5 files changed, 206 insertions(+), 1 deletion(-) create mode 100644 assembly/README create mode 100644 assembly/pom.xml create mode 100644 assembly/src/main/assembly/assembly.xml diff --git a/assembly/README b/assembly/README new file mode 100644 index 0000000000..6ee2a536d7 --- /dev/null +++ b/assembly/README @@ -0,0 +1,13 @@ +This is an assembly module for Spark project. + +It creates a single tar.gz file that includes all needed dependency of the project +except for org.apache.hadoop.* jars that are supposed to be available from the +deployed Hadoop cluster. + +This module is off by default to avoid spending extra time on top of repl-bin +module. To activate it specify the profile in the command line + -Passembly + +In case you want to avoid building time-expensive repl-bin module, that shaders +all the dependency into a big flat jar supplement maven command with + -DnoExpensive diff --git a/assembly/pom.xml b/assembly/pom.xml new file mode 100644 index 0000000000..1382539f24 --- /dev/null +++ b/assembly/pom.xml @@ -0,0 +1,104 @@ + + + 4.0.0 + + org.spark-project + spark-parent + 0.8.0-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-assembly + Spark Project Assembly + http://spark-project.org/ + + + + + org.apache.maven.plugins + maven-assembly-plugin + 2.4 + + + dist + package + + single + + + + src/main/assembly/assembly.xml + + + + + + + + + + + hadoop1 + + hadoop1 + + + + hadoop2 + + hadoop2 + + + + hadoop2-yarn + + hadoop2-yarn + + + + + + org.spark-project + spark-core + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-bagel + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-examples + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-examples + javadoc + 0.8.0-SNAPSHOT + + + org.spark-project + spark-examples + sources + 0.8.0-SNAPSHOT + + + org.spark-project + spark-repl + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-streaming + ${classifier.name} + 0.8.0-SNAPSHOT + + + \ No newline at end of file diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml new file mode 100644 index 0000000000..dd05f35f1f --- /dev/null +++ b/assembly/src/main/assembly/assembly.xml @@ -0,0 +1,68 @@ + + dist + + tar.gz + dir + + false + + + + + README + + + + + ${project.parent.basedir}/core/src/main/resources/spark/ui/static/ + + /ui-resources/spark/ui/static + + **/* + + + + + ${project.parent.basedir}/bin/ + + /bin + + **/* + + + + + ${project.parent.basedir} + + /bin + + run* + spark-shell* + spark-executor* + + + + + + + + org.spark-project:*:jar + + + org.spark-project:spark-dist:jar + + + + lib + true + false + runtime + false + + org.apache.hadoop:*:jar + org.spark-project:*:jar + + + + + diff --git a/pom.xml b/pom.xml index 4b48072c6e..6d9437b69f 100644 --- a/pom.xml +++ b/pom.xml @@ -60,7 +60,6 @@ examples streaming repl - repl-bin @@ -618,5 +617,25 @@ + + assembly + + false + + + assembly + + + + expensive-modules + + + !noExpensive + + + + repl-bin + + diff --git a/spark-shell b/spark-shell index 31a4138124..62fc18550d 100755 --- a/spark-shell +++ b/spark-shell @@ -79,6 +79,7 @@ if [[ ! $? ]]; then saved_stty="" fi +export SPARK_LAUNCH_WITH_SCALA=${SPARK_LAUNCH_WITH_SCALA:-1} $FWDIR/run $OPTIONS spark.repl.Main "$@" # record the exit status lest it be overwritten: From 0337d88321f3681009de548ce10ba7e0ca8f1a58 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Sun, 21 Jul 2013 18:07:19 -0700 Subject: [PATCH 236/419] Add a public method getCachedRdds to SparkContext --- core/src/main/scala/spark/SparkContext.scala | 8 +++++++- core/src/test/scala/spark/RDDSuite.scala | 6 ++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index c01e315e35..1b46665d2c 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -546,6 +546,12 @@ class SparkContext( StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) } + /** + * Returns an immutable map of RDDs that have marked themselves as cached via cache() call. + * Note that this does not necessarily mean the caching or computation was successful. + */ + def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.asInstanceOf[Map[Int, RDD[_]]] + def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos } @@ -580,7 +586,7 @@ class SparkContext( case null | "file" => if (SparkHadoopUtil.isYarnMode()) { logWarning("local jar specified as parameter to addJar under Yarn mode") - return + return } env.httpFileServer.addJar(new File(uri.getPath)) case _ => path diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index cbddf4e523..ff2dcd72d8 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -90,15 +90,19 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("basic caching") { + val origCachedRdds = sc.getCachedRDDs.size val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) + // Should only result in one cached RDD + assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("caching with failures") { val onlySplit = new Partition { override def index: Int = 0 } var shouldFail = true + val origCachedRdds = sc.getCachedRDDs.size val rdd = new RDD[Int](sc, Nil) { override def getPartitions: Array[Partition] = Array(onlySplit) override val getDependencies = List[Dependency[_]]() @@ -110,12 +114,14 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } }.cache() + assert(sc.getCachedRDDs.size === origCachedRdds + 1) val thrown = intercept[Exception]{ rdd.collect() } assert(thrown.getMessage.contains("injected failure")) shouldFail = false assert(rdd.collect().toList === List(1, 2, 3, 4)) + assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("empty RDD") { From 8901f379c98d19d3145f0cb2ae678d1b0060a311 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 09:58:03 -0700 Subject: [PATCH 237/419] Fixed memory used/remaining/total bug --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index fa516a4a34..5fb75c60fd 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -30,10 +30,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val memUsed = storageStatusList.map(_.memUsed()).reduce(_+_) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used") + val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used/Memory total", + "Disk used") def execRow(kv: Seq[String]) = {kv(0)} @@ -52,8 +54,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
        • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
        • + {Utils.memoryBytesToString(memUsed)} Used + ({Utils.memoryBytesToString(maxMem)} Total)
        • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
        @@ -70,8 +72,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { def getExecInfo(a: Int): Seq[String] = { val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) - val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) + val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) + val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString Seq( From f649dabb4a3a57cb25a852808297fb718cbfedd4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Jul 2013 13:14:33 -0700 Subject: [PATCH 238/419] Fix bug: DoubleRDDFunctions.sampleStdev() computed non-sample stdev(). Update JavaDoubleRDD to add new methods and docs. Fixes SPARK-825. --- .../main/scala/spark/DoubleRDDFunctions.scala | 8 ++++- .../scala/spark/api/java/JavaDoubleRDD.scala | 33 ++++++++++++++----- core/src/test/scala/spark/JavaAPISuite.java | 2 ++ 3 files changed, 33 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala index 93ef097702..104168e61c 100644 --- a/core/src/main/scala/spark/DoubleRDDFunctions.scala +++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala @@ -54,7 +54,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * Compute the sample standard deviation of this RDD's elements (which corrects for bias in * estimating the standard deviation by dividing by N-1 instead of N). */ - def sampleStdev(): Double = stats().stdev + def sampleStdev(): Double = stats().sampleStdev + + /** + * Compute the sample variance of this RDD's elements (which corrects for bias in + * estimating the variance by dividing by N-1 instead of N). + */ + def sampleVariance(): Double = stats().sampleVariance /** (Experimental) Approximate operation to return the mean within a timeout. */ def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 392556f261..8ce7df6213 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -115,33 +115,48 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav // Double RDD functions - /** Return the sum of the elements in this RDD. */ + /** Add up the elements in this RDD. */ def sum(): Double = srdd.sum() - /** Return a [[spark.StatCounter]] describing the elements in this RDD. */ + /** + * Return a [[spark.util.StatCounter]] object that captures the mean, variance and count + * of the RDD's elements in one operation. + */ def stats(): StatCounter = srdd.stats() - /** Return the mean of the elements in this RDD. */ + /** Compute the mean of this RDD's elements. */ def mean(): Double = srdd.mean() - /** Return the variance of the elements in this RDD. */ + /** Compute the variance of this RDD's elements. */ def variance(): Double = srdd.variance() - /** Return the standard deviation of the elements in this RDD. */ + /** Compute the standard deviation of this RDD's elements. */ def stdev(): Double = srdd.stdev() + /** + * Compute the sample standard deviation of this RDD's elements (which corrects for bias in + * estimating the standard deviation by dividing by N-1 instead of N). + */ + def sampleStdev(): Double = srdd.sampleStdev() + + /** + * Compute the sample variance of this RDD's elements (which corrects for bias in + * estimating the standard variance by dividing by N-1 instead of N). + */ + def sampleVariance(): Double = srdd.sampleVariance() + /** Return the approximate mean of the elements in this RDD. */ def meanApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = srdd.meanApprox(timeout, confidence) - /** Return the approximate mean of the elements in this RDD. */ + /** (Experimental) Approximate operation to return the mean within a timeout. */ def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout) - /** Return the approximate sum of the elements in this RDD. */ + /** (Experimental) Approximate operation to return the sum within a timeout. */ def sumApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - - /** Return the approximate sum of the elements in this RDD. */ + + /** (Experimental) Approximate operation to return the sum within a timeout. */ def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) } diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index aaf03e683b..5e2bf2d231 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -326,7 +326,9 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(20/6.0, rdd.mean(), 0.01); Assert.assertEquals(20/6.0, rdd.mean(), 0.01); Assert.assertEquals(6.22222, rdd.variance(), 0.01); + Assert.assertEquals(7.46667, rdd.sampleVariance(), 0.01); Assert.assertEquals(2.49444, rdd.stdev(), 0.01); + Assert.assertEquals(2.73252, rdd.sampleStdev(), 0.01); Double first = rdd.first(); List take = rdd.take(5); From 85c4d7bf3bf0969f58ebcda6ca68719972ff0c46 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 14:35:47 -0700 Subject: [PATCH 239/419] Shows number of complete/total/failed tasks (bug: failed tasks assigned to null executor) --- core/src/main/scala/spark/ui/SparkUI.scala | 1 + .../scala/spark/ui/exec/ExecutorsUI.scala | 65 ++++++++++++++++--- 2 files changed, 56 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 1534705bc5..7599f82a94 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -67,6 +67,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { // This server must register all handlers, including JobProgressUI, before binding // JobProgressUI registers a listener with SparkContext, which requires sc to initialize jobs.start() + exec.start() } def stop() { diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 5fb75c60fd..a981c680d2 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -5,15 +5,18 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.Properties -import spark.scheduler.cluster.TaskInfo +import spark.{ExceptionFailure, Logging, SparkContext, Success, Utils} import spark.executor.TaskMetrics -import spark.ui.JettyUtils._ -import spark.ui.UIUtils.headerSparkPage -import spark.ui.Page.Executors -import spark.storage.{StorageStatus, StorageUtils} +import spark.scheduler.cluster.TaskInfo +import spark.scheduler._ import spark.SparkContext +import spark.storage.{StorageStatus, StorageUtils} +import spark.ui.JettyUtils._ +import spark.ui.Page.Executors +import spark.ui.UIUtils.headerSparkPage import spark.ui.UIUtils import spark.Utils @@ -21,6 +24,14 @@ import scala.xml.{Node, XML} private[spark] class ExecutorsUI(val sc: SparkContext) { + private var _listener: Option[ExecutorsListener] = None + def listener = _listener.get + + def start() { + _listener = Some(new ExecutorsListener) + sc.addSparkListener(listener) + } + def getHandlers = Seq[(String, Handler)]( ("/executors", (request: HttpServletRequest) => render(request)) ) @@ -29,13 +40,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val storageStatusList = sc.getExecutorStorageStatus val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) val memUsed = storageStatusList.map(_.memUsed()).reduce(_+_) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used/Memory total", - "Disk used") + val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", + "Tasks: Complete/Total") def execRow(kv: Seq[String]) = {kv(0)} @@ -43,6 +53,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {kv(2)} {kv(3)} {kv(4)} + {kv(5)} val execInfo = for (b <- 0 until storageStatusList.size) @@ -76,12 +87,46 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString + val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0) + val totalTasks = listener.executorToTaskInfos(a.toString).size + val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0) match { + case f if f > 0 => " (%s failed)".format(f) + case _ => "" + } + Seq( execId, hostPort, rddBlocks, - "%s/%s".format(memUsed, maxMem), - diskUsed + "%s / %s".format(memUsed, maxMem), + diskUsed, + "%s / %s".format(completedTasks, totalTasks) + failedTasks ) } + + private[spark] class ExecutorsListener extends SparkListener with Logging { + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToTaskInfos = + HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val eid = taskEnd.taskMetrics.executorId + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + logInfo("Executor %s has %s failed tasks.".format(eid, executorToTasksFailed(eid))) + (Some(e), e.metrics) + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + logInfo("Executor %s has %s completed tasks.".format(eid, executorToTasksComplete(eid))) + (None, Some(taskEnd.taskMetrics)) + } + val taskList = executorToTaskInfos.getOrElse( + eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskEnd.taskInfo, metrics, failureInfo)) + executorToTaskInfos(eid) = taskList + } + } } \ No newline at end of file From 2eea974795dfa2bb79e66496454f36cb499065b0 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 15:15:54 -0700 Subject: [PATCH 240/419] Executors UI now calls executor ID from TaskInfo instead of TaskMetrics --- core/src/main/scala/spark/executor/Executor.scala | 1 - core/src/main/scala/spark/executor/TaskMetrics.scala | 5 ----- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 4 +--- 3 files changed, 1 insertion(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 82e5f5a015..2e81151882 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -126,7 +126,6 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => m.hostname = Utils.localHostName - m.executorId = executorId m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index e14dc57501..3151627839 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -23,11 +23,6 @@ class TaskMetrics extends Serializable { */ var hostname: String = _ - /** - * Executor's ID the task runs on - */ - var executorId: String = _ - /** * Time taken on the executor to deserialize this task */ diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index a981c680d2..40bee325b2 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -111,16 +111,14 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val eid = taskEnd.taskMetrics.executorId + val eid = taskEnd.taskInfo.executorId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - logInfo("Executor %s has %s failed tasks.".format(eid, executorToTasksFailed(eid))) (Some(e), e.metrics) case _ => executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - logInfo("Executor %s has %s completed tasks.".format(eid, executorToTasksComplete(eid))) (None, Some(taskEnd.taskMetrics)) } val taskList = executorToTaskInfos.getOrElse( From 8e38e7723230c1ca9564ddb79087fd1c4326b351 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 22 Jul 2013 16:05:32 -0700 Subject: [PATCH 241/419] Fix a test that was using an outdated config setting --- core/src/test/scala/spark/ShuffleSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 3a56c26bef..752e4b85e6 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -33,9 +33,9 @@ import spark.rdd.ShuffledRDD import spark.SparkContext._ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { - test("groupByKey with compression") { + test("groupByKey without compression") { try { - System.setProperty("spark.shuffle.compress", "true") + System.setProperty("spark.shuffle.compress", "false") sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) val groups = pairs.groupByKey(4).collect() @@ -45,7 +45,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val valuesFor2 = groups.find(_._1 == 2).get._2 assert(valuesFor2.toList.sorted === List(1)) } finally { - System.setProperty("spark.blockManager.compress", "false") + System.setProperty("spark.shuffle.compress", "true") } } From c83680434bfd6c241ed2e66f2e8704d3b93f752f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 17 Jul 2013 22:59:00 -0700 Subject: [PATCH 242/419] Add JavaAPICompletenessChecker. This is used to find methods in the Scala API that need to be ported to the Java API. To use it: ./run spark.tools.JavaAPICompletenessChecker Conflicts: project/SparkBuild.scala run run2.cmd --- bin/compute-classpath.cmd | 2 + bin/compute-classpath.sh | 2 + pom.xml | 1 + project/SparkBuild.scala | 8 +- tools/pom.xml | 63 +++ .../tools/JavaAPICompletenessChecker.scala | 359 ++++++++++++++++++ 6 files changed, 434 insertions(+), 1 deletion(-) create mode 100644 tools/pom.xml create mode 100644 tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 835d1af794..eb836b0ffd 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -33,6 +33,7 @@ set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel set MLLIB_DIR=%FWDIR%mllib +set TOOLS_DIR=%FWDIR%tools set STREAMING_DIR=%FWDIR%streaming set PYSPARK_DIR=%FWDIR%python @@ -48,6 +49,7 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes set CLASSPATH=%CLASSPATH%;%MLLIB_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%TOOLS_DIR%\target\scala-%SCALA_VERSION%\classes rem Add hadoop conf dir - else FileSystem.*, etc fail rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index e6fdc5306e..eb270a5428 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -36,6 +36,7 @@ REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" MLLIB_DIR="$FWDIR/mllib" +TOOLS_DIR="$FWDIR/tools" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -70,6 +71,7 @@ function dev_classpath { fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$TOOLS_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH="$CLASSPATH:$jar" done diff --git a/pom.xml b/pom.xml index 4b48072c6e..eb7bd7e9df 100644 --- a/pom.xml +++ b/pom.xml @@ -58,6 +58,7 @@ core bagel examples + tools streaming repl repl-bin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2f2cbf646a..f3f67b57c8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -40,7 +40,7 @@ object SparkBuild extends Build { //val HADOOP_MAJOR_VERSION = "2" //val HADOOP_YARN = true - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib, tools) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -48,6 +48,8 @@ object SparkBuild extends Build { lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) + lazy val tools = Project("tools", file("tools"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) + lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) @@ -233,6 +235,10 @@ object SparkBuild extends Build { ) ) + def toolsSettings = sharedSettings ++ Seq( + name := "spark-tools" + ) + def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") def mllibSettings = sharedSettings ++ Seq( diff --git a/tools/pom.xml b/tools/pom.xml new file mode 100644 index 0000000000..3dfba5808c --- /dev/null +++ b/tools/pom.xml @@ -0,0 +1,63 @@ + + + + 4.0.0 + + org.spark-project + spark-parent + 0.7.4-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-tools_2.9.3 + jar + Spark Project Tools + http://spark-project.org/ + + + + org.spark-project + spark-core_2.9.3 + ${project.version} + + + org.spark-project + spark-streaming_2.9.3 + ${project.version} + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.apache.maven.plugins + maven-source-plugin + + + org.codehaus.mojo + build-helper-maven-plugin + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala new file mode 100644 index 0000000000..89ef7236f4 --- /dev/null +++ b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.tools + +import spark._ +import java.lang.reflect.Method +import scala.collection.mutable.ArrayBuffer +import spark.api.java._ +import spark.streaming.{PairDStreamFunctions, DStream, StreamingContext} +import spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext} +import scala.Tuple2 + + +private[spark] abstract class SparkType(val name: String) + +private[spark] case class BaseType(override val name: String) extends SparkType(name) { + override def toString: String = { + name + } +} + +private[spark] +case class ParameterizedType(override val name: String, + parameters: Seq[SparkType], + typebounds: String = "") extends SparkType(name) { + override def toString: String = { + if (typebounds != "") { + typebounds + " " + name + "<" + parameters.mkString(", ") + ">" + } else { + name + "<" + parameters.mkString(", ") + ">" + } + } +} + +private[spark] +case class SparkMethod(name: String, returnType: SparkType, parameters: Seq[SparkType]) { + override def toString: String = { + returnType + " " + name + "(" + parameters.mkString(", ") + ")" + } +} + +/** + * A tool for identifying methods that need to be ported from Scala to the Java API. + * + * It uses reflection to find methods in the Scala API and rewrites those methods' signatures + * into appropriate Java equivalents. If those equivalent methods have not been implemented in + * the Java API, they are printed. + */ +object JavaAPICompletenessChecker { + + private def parseType(typeStr: String): SparkType = { + if (!typeStr.contains("<")) { + // Base types might begin with "class" or "interface", so we have to strip that off: + BaseType(typeStr.trim.split(" ").last) + } else if (typeStr.endsWith("[]")) { + ParameterizedType("Array", Seq(parseType(typeStr.stripSuffix("[]")))) + } else { + val parts = typeStr.split("<", 2) + val name = parts(0).trim + assert (parts(1).last == '>') + val parameters = parts(1).dropRight(1) + ParameterizedType(name, parseTypeList(parameters)) + } + } + + private def parseTypeList(typeStr: String): Seq[SparkType] = { + val types: ArrayBuffer[SparkType] = new ArrayBuffer[SparkType] + var stack = 0 + var token: StringBuffer = new StringBuffer() + for (c <- typeStr.trim) { + if (c == ',' && stack == 0) { + types += parseType(token.toString) + token = new StringBuffer() + } else if (c == ' ' && stack != 0) { + // continue + } else { + if (c == '<') { + stack += 1 + } else if (c == '>') { + stack -= 1 + } + token.append(c) + } + } + assert (stack == 0) + if (token.toString != "") { + types += parseType(token.toString) + } + types.toSeq + } + + private def parseReturnType(typeStr: String): SparkType = { + if (typeStr(0) == '<') { + val parts = typeStr.drop(0).split(">", 2) + val parsed = parseType(parts(1)).asInstanceOf[ParameterizedType] + ParameterizedType(parsed.name, parsed.parameters, parts(0)) + } else { + parseType(typeStr) + } + } + + private def toSparkMethod(method: Method): SparkMethod = { + val returnType = parseReturnType(method.getGenericReturnType.toString) + val name = method.getName + val parameters = method.getGenericParameterTypes.map(t => parseType(t.toString)) + SparkMethod(name, returnType, parameters) + } + + private def toJavaType(scalaType: SparkType): SparkType = { + val renameSubstitutions = Map( + "scala.collection.Map" -> "java.util.Map", + // TODO: the JavaStreamingContext API accepts Array arguments + // instead of Lists, so this isn't a trivial translation / sub: + "scala.collection.Seq" -> "java.util.List", + "scala.Function2" -> "spark.api.java.function.Function2", + "scala.collection.Iterator" -> "java.util.Iterator", + "scala.collection.mutable.Queue" -> "java.util.Queue", + "double" -> "java.lang.Double" + ) + // Keep applying the substitutions until we've reached a fixedpoint. + def applySubs(scalaType: SparkType): SparkType = { + scalaType match { + case ParameterizedType(name, parameters, typebounds) => + name match { + case "spark.RDD" => + if (parameters(0).name == classOf[Tuple2[_, _]].getName) { + val tupleParams = + parameters(0).asInstanceOf[ParameterizedType].parameters.map(toJavaType) + ParameterizedType(classOf[JavaPairRDD[_, _]].getName, tupleParams) + } else { + ParameterizedType(classOf[JavaRDD[_]].getName, parameters.map(toJavaType)) + } + case "spark.streaming.DStream" => + if (parameters(0).name == classOf[Tuple2[_, _]].getName) { + val tupleParams = + parameters(0).asInstanceOf[ParameterizedType].parameters.map(toJavaType) + ParameterizedType("spark.streaming.api.java.JavaPairDStream", tupleParams) + } else { + ParameterizedType("spark.streaming.api.java.JavaDStream", + parameters.map(toJavaType)) + } + // TODO: Spark Streaming uses Guava's Optional in place of Option, leading to some + // false-positives here: + case "scala.Option" => + toJavaType(parameters(0)) + case "scala.Function1" => + val firstParamName = parameters.last.name + if (firstParamName.startsWith("scala.collection.Traversable") || + firstParamName.startsWith("scala.collection.Iterator")) { + ParameterizedType("spark.api.java.function.FlatMapFunction", + Seq(parameters(0), + parameters.last.asInstanceOf[ParameterizedType].parameters(0)).map(toJavaType)) + } else if (firstParamName == "scala.runtime.BoxedUnit") { + ParameterizedType("spark.api.java.function.VoidFunction", + parameters.dropRight(1).map(toJavaType)) + } else { + ParameterizedType("spark.api.java.function.Function", parameters.map(toJavaType)) + } + case _ => + ParameterizedType(renameSubstitutions.getOrElse(name, name), + parameters.map(toJavaType)) + } + case BaseType(name) => + if (renameSubstitutions.contains(name)) { + BaseType(renameSubstitutions(name)) + } else { + scalaType + } + } + } + var oldType = scalaType + var newType = applySubs(scalaType) + while (oldType != newType) { + oldType = newType + newType = applySubs(scalaType) + } + newType + } + + private def toJavaMethod(method: SparkMethod): SparkMethod = { + val params = method.parameters + .filterNot(_.name == "scala.reflect.ClassManifest").map(toJavaType) + SparkMethod(method.name, toJavaType(method.returnType), params) + } + + private def isExcludedByName(method: Method): Boolean = { + val name = method.getDeclaringClass.getName + "." + method.getName + // Scala methods that are declared as private[mypackage] become public in the resulting + // Java bytecode. As a result, we need to manually exclude those methods here. + // This list also includes a few methods that are only used by the web UI or other + // internal Spark components. + val excludedNames = Seq( + "spark.RDD.origin", + "spark.RDD.elementClassManifest", + "spark.RDD.checkpointData", + "spark.RDD.partitioner", + "spark.RDD.partitions", + "spark.RDD.firstParent", + "spark.RDD.doCheckpoint", + "spark.RDD.markCheckpointed", + "spark.RDD.clearDependencies", + "spark.RDD.getDependencies", + "spark.RDD.getPartitions", + "spark.RDD.dependencies", + "spark.RDD.getPreferredLocations", + "spark.RDD.collectPartitions", + "spark.RDD.computeOrReadCheckpoint", + "spark.PairRDDFunctions.getKeyClass", + "spark.PairRDDFunctions.getValueClass", + "spark.SparkContext.stringToText", + "spark.SparkContext.makeRDD", + "spark.SparkContext.runJob", + "spark.SparkContext.runApproximateJob", + "spark.SparkContext.clean", + "spark.SparkContext.metadataCleaner", + "spark.SparkContext.ui", + "spark.SparkContext.newShuffleId", + "spark.SparkContext.newRddId", + "spark.SparkContext.cleanup", + "spark.SparkContext.receiverJobThread", + "spark.SparkContext.getRDDStorageInfo", + "spark.SparkContext.addedFiles", + "spark.SparkContext.addedJars", + "spark.SparkContext.persistentRdds", + "spark.SparkContext.executorEnvs", + "spark.SparkContext.checkpointDir", + "spark.SparkContext.getSparkHome", + "spark.SparkContext.executorMemoryRequested", + "spark.SparkContext.getExecutorStorageStatus", + "spark.streaming.DStream.generatedRDDs", + "spark.streaming.DStream.zeroTime", + "spark.streaming.DStream.rememberDuration", + "spark.streaming.DStream.storageLevel", + "spark.streaming.DStream.mustCheckpoint", + "spark.streaming.DStream.checkpointDuration", + "spark.streaming.DStream.checkpointData", + "spark.streaming.DStream.graph", + "spark.streaming.DStream.isInitialized", + "spark.streaming.DStream.parentRememberDuration", + "spark.streaming.DStream.initialize", + "spark.streaming.DStream.validate", + "spark.streaming.DStream.setContext", + "spark.streaming.DStream.setGraph", + "spark.streaming.DStream.remember", + "spark.streaming.DStream.getOrCompute", + "spark.streaming.DStream.generateJob", + "spark.streaming.DStream.clearOldMetadata", + "spark.streaming.DStream.addMetadata", + "spark.streaming.DStream.updateCheckpointData", + "spark.streaming.DStream.restoreCheckpointData", + "spark.streaming.DStream.isTimeValid", + "spark.streaming.StreamingContext.nextNetworkInputStreamId", + "spark.streaming.StreamingContext.networkInputTracker", + "spark.streaming.StreamingContext.checkpointDir", + "spark.streaming.StreamingContext.checkpointDuration", + "spark.streaming.StreamingContext.receiverJobThread", + "spark.streaming.StreamingContext.scheduler", + "spark.streaming.StreamingContext.initialCheckpoint", + "spark.streaming.StreamingContext.getNewNetworkStreamId", + "spark.streaming.StreamingContext.validate", + "spark.streaming.StreamingContext.createNewSparkContext", + "spark.streaming.StreamingContext.rddToFileName", + "spark.streaming.StreamingContext.getSparkCheckpointDir", + "spark.streaming.StreamingContext.env", + "spark.streaming.StreamingContext.graph", + "spark.streaming.StreamingContext.isCheckpointPresent" + ) + val excludedPatterns = Seq( + """^spark\.SparkContext\..*To.*Functions""", + """^spark\.SparkContext\..*WritableConverter""", + """^spark\.SparkContext\..*To.*Writable""" + ).map(_.r) + lazy val excludedByPattern = + !excludedPatterns.map(_.findFirstIn(name)).filter(_.isDefined).isEmpty + name.contains("$") || excludedNames.contains(name) || excludedByPattern + } + + private def isExcludedByAnnotation(method: Method): Boolean = { + method.getAnnotation(classOf[ExcludeFromJavaAPI]) != null + } + + private def isExcludedByInterface(method: Method): Boolean = { + val excludedInterfaces = + Set("spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") + def toComparisionKey(method: Method) = + (method.getReturnType, method.getName, method.getGenericReturnType) + val interfaces = method.getDeclaringClass.getInterfaces.filter { i => + excludedInterfaces.contains(i.getName) + } + val excludedMethods = interfaces.flatMap(_.getMethods.map(toComparisionKey)) + excludedMethods.contains(toComparisionKey(method)) + } + + private def printMissingMethods(scalaClass: Class[_], javaClass: Class[_]) { + val methods = scalaClass.getMethods + .filterNot(_.isAccessible) + .filterNot(isExcludedByAnnotation) + .filterNot(isExcludedByName) + .filterNot(isExcludedByInterface) + val javaEquivalents = methods.map(m => toJavaMethod(toSparkMethod(m))).toSet + + val javaMethods = javaClass.getMethods.map(toSparkMethod).toSet + + val missingMethods = javaEquivalents -- javaMethods + + for (method <- missingMethods) { + println(method) + } + } + + def main(args: Array[String]) { + println("Missing RDD methods") + printMissingMethods(classOf[RDD[_]], classOf[JavaRDD[_]]) + println() + + println("Missing PairRDD methods") + printMissingMethods(classOf[PairRDDFunctions[_, _]], classOf[JavaPairRDD[_, _]]) + println() + + println("Missing DoubleRDD methods") + printMissingMethods(classOf[DoubleRDDFunctions], classOf[JavaDoubleRDD]) + println() + + println("Missing OrderedRDD methods") + printMissingMethods(classOf[OrderedRDDFunctions[_, _]], classOf[JavaPairRDD[_, _]]) + println() + + println("Missing SparkContext methods") + printMissingMethods(classOf[SparkContext], classOf[JavaSparkContext]) + println() + + println("Missing StreamingContext methods") + printMissingMethods(classOf[StreamingContext], classOf[JavaStreamingContext]) + println() + + println("Missing DStream methods") + printMissingMethods(classOf[DStream[_]], classOf[JavaDStream[_]]) + println() + + println("Missing PairDStream methods") + printMissingMethods(classOf[PairDStreamFunctions[_, _]], classOf[JavaPairDStream[_, _]]) + println() + } +} From e17e1b388eaddea39a7272bae201fd3d0060c821 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 20 Jul 2013 22:04:30 -0700 Subject: [PATCH 243/419] Remove annotation code that broke build. --- .../main/scala/spark/tools/JavaAPICompletenessChecker.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala index 89ef7236f4..3a55f50812 100644 --- a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala @@ -290,10 +290,6 @@ object JavaAPICompletenessChecker { name.contains("$") || excludedNames.contains(name) || excludedByPattern } - private def isExcludedByAnnotation(method: Method): Boolean = { - method.getAnnotation(classOf[ExcludeFromJavaAPI]) != null - } - private def isExcludedByInterface(method: Method): Boolean = { val excludedInterfaces = Set("spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") @@ -309,7 +305,6 @@ object JavaAPICompletenessChecker { private def printMissingMethods(scalaClass: Class[_], javaClass: Class[_]) { val methods = scalaClass.getMethods .filterNot(_.isAccessible) - .filterNot(isExcludedByAnnotation) .filterNot(isExcludedByName) .filterNot(isExcludedByInterface) val javaEquivalents = methods.map(m => toJavaMethod(toSparkMethod(m))).toSet From 872c97ad829ba20e866c4e45054e7d2d05b02042 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 16:54:37 -0700 Subject: [PATCH 244/419] Split task columns, memory columns sort by numeric value --- .../scala/spark/ui/exec/ExecutorsUI.scala | 36 +++++++++++-------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 40bee325b2..20ea54d6a6 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -45,15 +45,21 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { .reduceOption(_+_).getOrElse(0L) val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", - "Tasks: Complete/Total") + "Failed tasks", "Complete tasks", "Total tasks") def execRow(kv: Seq[String]) = {kv(0)} {kv(1)} {kv(2)} - {kv(3)} - {kv(4)} - {kv(5)} + + {Utils.memoryBytesToString(kv(3).toLong)} / {Utils.memoryBytesToString(kv(4).toLong)} + + + {Utils.memoryBytesToString(kv(5).toLong)} + + {kv(6)} + {kv(7)} + {kv(8)} val execInfo = for (b <- 0 until storageStatusList.size) @@ -83,24 +89,24 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { def getExecInfo(a: Int): Seq[String] = { val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) - val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) - val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString - val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0) - val totalTasks = listener.executorToTaskInfos(a.toString).size - val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0) match { - case f if f > 0 => " (%s failed)".format(f) - case _ => "" - } + val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString + val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString + val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString + val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString + val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString + val totalTasks = listener.executorToTaskInfos(a.toString).size.toString Seq( execId, hostPort, rddBlocks, - "%s / %s".format(memUsed, maxMem), + memUsed, + maxMem, diskUsed, - "%s / %s".format(completedTasks, totalTasks) + failedTasks + failedTasks, + completedTasks, + totalTasks ) } From 2c2bfbe294c0082520c80a01562a2dbeeba63b7a Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 01:36:44 -0700 Subject: [PATCH 245/419] Add toMap method to TimeStampedHashMap and use it --- core/src/main/scala/spark/SparkContext.scala | 2 +- core/src/main/scala/spark/util/TimeStampedHashMap.scala | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1b46665d2c..0fb7dfa810 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -550,7 +550,7 @@ class SparkContext( * Returns an immutable map of RDDs that have marked themselves as cached via cache() call. * Note that this does not necessarily mean the caching or computation was successful. */ - def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.asInstanceOf[Map[Int, RDD[_]]] + def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.toMap def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index cc7909194a..07772a0afb 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -20,6 +20,7 @@ package spark.util import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Map +import scala.collection.immutable import spark.scheduler.MapStatus /** @@ -99,6 +100,8 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging { } } + def toMap: immutable.Map[A, B] = iterator.toMap + /** * Removes old key-value pairs that have timestamp earlier than `threshTime` */ From 4830e225624091fa836012651420cf2b5b97dcca Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 09:50:13 -0700 Subject: [PATCH 246/419] Rename method per rxin feedback --- core/src/main/scala/spark/SparkContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 0fb7dfa810..24ba605646 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -547,10 +547,10 @@ class SparkContext( } /** - * Returns an immutable map of RDDs that have marked themselves as cached via cache() call. + * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call. * Note that this does not necessarily mean the caching or computation was successful. */ - def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.toMap + def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos From efd6418c1b99c1ecc2b0a4c72e6430eea4d86260 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 10:40:41 -0700 Subject: [PATCH 247/419] Move getPersistentRDDs testing to a new Suite --- core/src/test/scala/spark/RDDSuite.scala | 6 -- .../scala/spark/SparkContextInfoSuite.scala | 60 +++++++++++++++++++ 2 files changed, 60 insertions(+), 6 deletions(-) create mode 100644 core/src/test/scala/spark/SparkContextInfoSuite.scala diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index ff2dcd72d8..cbddf4e523 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -90,19 +90,15 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("basic caching") { - val origCachedRdds = sc.getCachedRDDs.size val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) - // Should only result in one cached RDD - assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("caching with failures") { val onlySplit = new Partition { override def index: Int = 0 } var shouldFail = true - val origCachedRdds = sc.getCachedRDDs.size val rdd = new RDD[Int](sc, Nil) { override def getPartitions: Array[Partition] = Array(onlySplit) override val getDependencies = List[Dependency[_]]() @@ -114,14 +110,12 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } }.cache() - assert(sc.getCachedRDDs.size === origCachedRdds + 1) val thrown = intercept[Exception]{ rdd.collect() } assert(thrown.getMessage.contains("injected failure")) shouldFail = false assert(rdd.collect().toList === List(1, 2, 3, 4)) - assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("empty RDD") { diff --git a/core/src/test/scala/spark/SparkContextInfoSuite.scala b/core/src/test/scala/spark/SparkContextInfoSuite.scala new file mode 100644 index 0000000000..6d50bf5e1b --- /dev/null +++ b/core/src/test/scala/spark/SparkContextInfoSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark + +import org.scalatest.FunSuite +import spark.SparkContext._ + +class SparkContextInfoSuite extends FunSuite with LocalSparkContext { + test("getPersistentRDDs only returns RDDs that are marked as cached") { + sc = new SparkContext("local", "test") + assert(sc.getPersistentRDDs.isEmpty === true) + + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) + assert(sc.getPersistentRDDs.isEmpty === true) + + rdd.cache() + assert(sc.getPersistentRDDs.size === 1) + assert(sc.getPersistentRDDs.values.head === rdd) + } + + test("getPersistentRDDs returns an immutable map") { + sc = new SparkContext("local", "test") + val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + + val myRdds = sc.getPersistentRDDs + assert(myRdds.size === 1) + assert(myRdds.values.head === rdd1) + + val rdd2 = sc.makeRDD(Array(5, 6, 7, 8), 1).cache() + + // getPersistentRDDs should have 2 RDDs, but myRdds should not change + assert(sc.getPersistentRDDs.size === 2) + assert(myRdds.size === 1) + } + + test("getRDDStorageInfo only reports on RDDs that actually persist data") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + + assert(sc.getRDDStorageInfo.size === 0) + + rdd.collect() + assert(sc.getRDDStorageInfo.size === 1) + } +} \ No newline at end of file From 87a9dd898ff51fd110799edae087d59f6b714211 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 12:13:27 -0700 Subject: [PATCH 248/419] Made RegressionModel serializable and added unit tests to make sure predict methods would work. --- .../mllib/optimization/GradientDescent.scala | 6 ++-- .../spark/mllib/optimization/Updater.scala | 6 ++-- .../mllib/regression/LogisticRegression.scala | 6 +++- .../spark/mllib/regression/Regression.scala | 2 +- .../mllib/regression/RidgeRegression.scala | 5 ++- .../regression/LogisticRegressionSuite.scala | 33 +++++++++++++++---- 6 files changed, 42 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 4c996c0903..185a2a24f6 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -39,9 +39,9 @@ object GradientDescent { * @param miniBatchFraction - fraction of the input data set that should be used for * one iteration of SGD. Default value 1.0. * - * @return weights - Column matrix containing weights for every feature. - * @return stochasticLossHistory - Array containing the stochastic loss computed for - * every iteration. + * @return A tuple containing two elements. The first element is a column matrix containing + * weights for every feature, and the second element is an array containing the stochastic + * loss computed for every iteration. */ def runMiniBatchSGD( data: RDD[(Double, Array[Double])], diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index b864fd4634..18cb5f3a95 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -23,13 +23,13 @@ abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize and iteration number. * - * @param weightsOld - Column matrix of size nx1 where n is the number of features. + * @param weightsOlds - Column matrix of size nx1 where n is the number of features. * @param gradient - Column matrix of size nx1 where n is the number of features. * @param stepSize - step size across iterations * @param iter - Iteration number * - * @return weightsNew - Column matrix containing updated weights - * @return reg_val - regularization value + * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, + * and the second element is the regularization value. */ def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): (DoubleMatrix, Double) diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index 711e205c39..4b22546017 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -36,8 +36,12 @@ class LogisticRegressionModel( private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) override def predict(testData: spark.RDD[Array[Double]]) = { + // A small optimization to avoid serializing the entire model. Only the weightsMatrix + // and intercept is needed. + val localWeights = weightsMatrix + val localIntercept = intercept testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(weightsMatrix).get(0) + this.intercept + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept 1.0/ (1.0 + math.exp(margin * -1)) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala index 645204ddf3..b845ba1a89 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Regression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Regression.scala @@ -19,7 +19,7 @@ package spark.mllib.regression import spark.RDD -trait RegressionModel { +trait RegressionModel extends Serializable { /** * Predict values for the given data set using the model trained. * diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index f724edd732..6ba141e8fb 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -37,8 +37,11 @@ class RidgeRegressionModel( extends RegressionModel { override def predict(testData: RDD[Array[Double]]): RDD[Double] = { + // A small optimization to avoid serializing the entire model. + val localIntercept = this.intercept + val localWeights = this.weights testData.map { x => - (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + (new DoubleMatrix(1, x.length, x:_*).mmul(localWeights)).get(0) + localIntercept } } diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 47191d9a5a..6a8098b59d 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { @@ -51,15 +50,24 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => + val y: Seq[Double] = (0 until nPoints).map { i => val yVal = offset + scale * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))) + val testData = (0 until nPoints).map(i => (y(i), Array(x1(i)))) testData } + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + val offPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(offPredictions < input.length / 5) + } + // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("logistic regression") { val nPoints = 10000 @@ -70,14 +78,20 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(10.0) - .setNumIterations(20) + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(20) val model = lr.train(testRDD) + // Test the weights val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + // Test prediction on RDD. + validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + + // Test prediction on Array. + validatePrediction(testData.map(row => model.predict(row._2)), testData) } test("logistic regression with initial weights") { @@ -94,13 +108,18 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { testRDD.cache() // Use half as many iterations as the previous test. - val lr = new LogisticRegression().setStepSize(10.0) - .setNumIterations(10) + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(10) val model = lr.train(testRDD, initialWeights) val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + // Test prediction on RDD. + validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + + // Test prediction on Array. + validatePrediction(testData.map(row => model.predict(row._2)), testData) } } From 2210e8ccf8d77f65442a344c4eae39e000fba927 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 12:52:15 -0700 Subject: [PATCH 249/419] Use a different validation dataset for Logistic Regression prediction testing. --- .../regression/LogisticRegressionSuite.scala | 29 +++++++++++-------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 6a8098b59d..0a99b78cf8 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -35,10 +35,11 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { // Generate input of the form Y = logistic(offset + scale*X) def generateLogisticInput( - offset: Double, - scale: Double, - nPoints: Int) : Seq[(Double, Array[Double])] = { - val rnd = new Random(42) + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) @@ -60,12 +61,12 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { } def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { - val offPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => // A prediction is off if the prediction is more than 0.5 away from expected value. math.abs(prediction - expected) > 0.5 }.size // At least 80% of the predictions should be on. - assert(offPredictions < input.length / 5) + assert(numOffPredictions < input.length / 5) } // Test if we can correctly learn A, B where Y = logistic(A + B*X) @@ -74,7 +75,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val A = 2.0 val B = -1.5 - val testData = generateLogisticInput(A, B, nPoints) + val testData = generateLogisticInput(A, B, nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() @@ -87,11 +88,13 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. - validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) // Test prediction on Array. - validatePrediction(testData.map(row => model.predict(row._2)), testData) + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } test("logistic regression with initial weights") { @@ -99,7 +102,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val A = 2.0 val B = -1.5 - val testData = generateLogisticInput(A, B, nPoints) + val testData = generateLogisticInput(A, B, nPoints, 42) val initialB = -1.0 val initialWeights = Array(initialB) @@ -116,10 +119,12 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. - validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) // Test prediction on Array. - validatePrediction(testData.map(row => model.predict(row._2)), testData) + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } From 0200801a55b580c7504687e3476b7a71c7699001 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 13:35:43 -0700 Subject: [PATCH 250/419] Tracks task start events and shows number of active tasks on Executor UI --- .../scala/spark/scheduler/DAGScheduler.scala | 8 ++++++++ .../spark/scheduler/DAGSchedulerEvent.scala | 2 ++ .../main/scala/spark/scheduler/JobLogger.scala | 16 +++++++++++++++- .../scala/spark/scheduler/SparkListener.scala | 9 ++++++++- .../spark/scheduler/TaskSchedulerListener.scala | 3 +++ .../cluster/ClusterTaskSetManager.scala | 6 ++++++ .../scheduler/local/LocalTaskSetManager.scala | 5 +++++ .../main/scala/spark/ui/exec/ExecutorsUI.scala | 12 +++++++++++- 8 files changed, 58 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 29e879aa42..b02bf8f4bf 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -52,6 +52,11 @@ class DAGScheduler( } taskSched.setListener(this) + //Called by TaskScheduler to report task's starting. + override def taskStarted(task: Task[_], taskInfo: TaskInfo) { + eventQueue.put(BeginEvent(task, taskInfo)) + } + // Called by TaskScheduler to report task completions or failures. override def taskEnded( task: Task[_], @@ -343,6 +348,9 @@ class DAGScheduler( case ExecutorLost(execId) => handleExecutorLost(execId) + case begin: BeginEvent => + sparkListeners.foreach(_.onTaskStart(SparkListenerTaskStart(begin.task, begin.taskInfo))) + case completion: CompletionEvent => sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion.task, completion.reason, completion.taskInfo, completion.taskMetrics))) diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 506c87f65b..3b4ee6287a 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -43,6 +43,8 @@ private[spark] case class JobSubmitted( properties: Properties = null) extends DAGSchedulerEvent +private[spark] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent + private[spark] case class CompletionEvent( task: Task[_], reason: TaskEndReason, diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 85b5ddd4a8..f7565b8c57 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -68,6 +68,8 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { processStageCompletedEvent(stageInfo) case SparkListenerJobEnd(job, result) => processJobEndEvent(job, result) + case SparkListenerTaskStart(task, taskInfo) => + processTaskStartEvent(task, taskInfo) case SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics) => processTaskEndEvent(task, reason, taskInfo, taskMetrics) case _ => @@ -252,7 +254,19 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { stageInfo.stage.id + " STATUS=COMPLETED") } - + + override def onTaskStart(taskStart: SparkListenerTaskStart) { + eventQueue.put(taskStart) + } + + protected def processTaskStartEvent(task: Task[_], taskInfo: TaskInfo) { + var taskStatus = "" + task match { + case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" + case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" + } + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { eventQueue.put(taskEnd) } diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 4fb1c5d42d..4eb7e4e6a5 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -29,6 +29,8 @@ case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends Spar case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents +case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents + case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvents @@ -48,7 +50,12 @@ trait SparkListener { * Called when a stage is submitted */ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } - + + /** + * Called when a task starts + */ + def onTaskStart(taskEnd: SparkListenerTaskStart) { } + /** * Called when a task ends */ diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index 245e7ccb52..2cdeb1c8c0 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -27,6 +27,9 @@ import spark.executor.TaskMetrics * Interface for getting events back from the TaskScheduler. */ private[spark] trait TaskSchedulerListener { + // A task has started. + def taskStarted(task: Task[_], taskInfo: TaskInfo) + // A task has finished or failed. def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 3d06520675..14e87af653 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -496,6 +496,8 @@ private[spark] class ClusterTaskSetManager( 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) + if (taskAttempts(index).size == 1) + taskStarted(task,info) return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) } case _ => @@ -518,6 +520,10 @@ private[spark] class ClusterTaskSetManager( } } + def taskStarted(task: Task[_], info: TaskInfo) { + sched.listener.taskStarted(task, info) + } + def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { val info = taskInfos(tid) if (info.failed) { diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index e662ad6709..b500451990 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -117,6 +117,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val taskName = "task %s:%d".format(taskSet.id, index) copiesRunning(index) += 1 increaseRunningTasks(1) + taskStarted(task, info) return Some(new TaskDescription(taskId, null, taskName, bytes)) case None => {} } @@ -146,6 +147,10 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } } + def taskStarted(task: Task[_], info: TaskInfo) { + sched.listener.taskStarted(task, info) + } + def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { val info = taskInfos(tid) val index = info.index diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 20ea54d6a6..9ac33326c0 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -45,7 +45,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { .reduceOption(_+_).getOrElse(0L) val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", - "Failed tasks", "Complete tasks", "Total tasks") + "Active tasks", "Failed tasks", "Complete tasks", "Total tasks") def execRow(kv: Seq[String]) = {kv(0)} @@ -60,6 +60,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {kv(6)} {kv(7)} {kv(8)} + {kv(9)} val execInfo = for (b <- 0 until storageStatusList.size) @@ -93,6 +94,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString + val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, 0).toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString @@ -104,6 +106,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { memUsed, maxMem, diskUsed, + activeTasks, failedTasks, completedTasks, totalTasks @@ -111,13 +114,20 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { + val executorToTasksActive = HashMap[String, Int]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + override def onTaskStart(taskStart: SparkListenerTaskStart) { + val eid = taskStart.taskInfo.executorId + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) - 1 val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => From 9f2dbb2a7ca9c1cf878cf96df7122b05d95e799b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:10:09 -0700 Subject: [PATCH 251/419] Adds/removes active tasks only once --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 9ac33326c0..bb2b003486 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -94,7 +94,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, 0).toString + val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, Seq[Long]()).size.toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString @@ -114,7 +114,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, Int]() + val executorToTasksActive = HashMap[String, Seq[Long]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = @@ -122,12 +122,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()) :+ taskStart.taskInfo.taskId } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) - 1 + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()).filterNot(_ == taskEnd.taskInfo.taskId) val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => From 101b8cc78a1fb49332265d7a7e4bf4ec6a22dd42 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:20 -0700 Subject: [PATCH 252/419] SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure. --- .../scala/spark/scheduler/DAGScheduler.scala | 29 +++++++++++++------ core/src/test/scala/spark/FailureSuite.scala | 29 ++++++++++++++----- 2 files changed, 42 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 29e879aa42..5fcd807aff 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -17,19 +17,17 @@ package spark.scheduler -import cluster.TaskInfo -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.LinkedBlockingQueue -import java.util.concurrent.TimeUnit +import java.io.NotSerializableException import java.util.Properties +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ import spark.executor.TaskMetrics -import spark.partial.ApproximateActionListener -import spark.partial.ApproximateEvaluator -import spark.partial.PartialResult +import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} +import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -258,7 +256,8 @@ class DAGScheduler( assert(partitions.size > 0) val waiter = new JobWaiter(partitions.size, resultHandler) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) + val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, + properties) return (toSubmit, waiter) } @@ -283,7 +282,7 @@ class DAGScheduler( "Total number of partitions: " + maxPartitions) } - val (toSubmit, waiter) = prepareJob( + val (toSubmit: JobSubmitted, waiter: JobWaiter[_]) = prepareJob( finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) waiter.awaitResult() match { @@ -466,6 +465,18 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") + + // Preemptively serialize the stage RDD to make sure the tasks for this stage will be + // serializable. We are catching this exception here because it would be fairly hard to + // catch the non-serializable exception down the road, where we have several different + // implementations for local scheduler and cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) + } catch { + case e: NotSerializableException => abortStage(stage, e.toString) + return + } + if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index 6c847b8fef..c3c52f9118 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -18,9 +18,6 @@ package spark import org.scalatest.FunSuite -import org.scalatest.prop.Checkers - -import scala.collection.mutable.ArrayBuffer import SparkContext._ @@ -40,7 +37,7 @@ object FailureSuiteState { } class FailureSuite extends FunSuite with LocalSparkContext { - + // Run a 3-task map job in which task 1 deterministically fails once, and check // whether the job completes successfully and we ran 4 tasks in total. test("failure in a single-stage job") { @@ -66,7 +63,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure in a two-stage job") { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map { - case (k, v) => + case (k, v) => FailureSuiteState.synchronized { FailureSuiteState.tasksRun += 1 if (k == 1 && FailureSuiteState.tasksFailed == 0) { @@ -87,15 +84,33 @@ class FailureSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => new NonSerializable) - val thrown = intercept[spark.SparkException] { + val thrown = intercept[SparkException] { results.collect() } - assert(thrown.getClass === classOf[spark.SparkException]) + assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) FailureSuiteState.clear() } + test("failure because task closure is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + val thrown = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => a).count() + } + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("NotSerializableException")) + + val thrown1 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() + } + assert(thrown1.getClass === classOf[SparkException]) + assert(thrown1.getMessage.contains("NotSerializableException")) + + FailureSuiteState.clear() + } + // TODO: Need to add tests with shuffle fetch failures. } From 5ed38b4d1d7154235d5b72c35d3a8e63bac3a2de Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:59 -0700 Subject: [PATCH 253/419] Scheduler code style cleanup. --- .../cluster/ClusterTaskSetManager.scala | 154 +++++++++++------- .../scheduler/cluster/TaskSetManager.scala | 22 ++- .../scheduler/local/LocalScheduler.scala | 22 ++- .../scheduler/local/LocalTaskSetManager.scala | 82 +++++----- .../scheduler/ClusterSchedulerSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 13 +- 6 files changed, 170 insertions(+), 125 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 3d06520675..d934293b70 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -17,7 +17,8 @@ package spark.scheduler.cluster -import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} +import java.nio.ByteBuffer +import java.util.{Arrays, NoSuchElementException} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -25,12 +26,14 @@ import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import spark._ -import spark.scheduler._ +import spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils} +import spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} import spark.TaskState.TaskState -import java.nio.ByteBuffer +import spark.scheduler.{ShuffleMapTask, Task, TaskResult, TaskSet} -private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { + +private[spark] object TaskLocality + extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { // process local is expected to be used ONLY within tasksetmanager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value @@ -43,8 +46,10 @@ private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LO assert (constraint != TaskLocality.PROCESS_LOCAL) constraint match { - case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL - case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL + case TaskLocality.NODE_LOCAL => + condition == TaskLocality.NODE_LOCAL + case TaskLocality.RACK_LOCAL => + condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL // For anything else, allow case _ => true } @@ -56,11 +61,10 @@ private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LO val retval = TaskLocality.withName(str) // Must not specify PROCESS_LOCAL ! assert (retval != TaskLocality.PROCESS_LOCAL) - retval } catch { case nEx: NoSuchElementException => { - logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL"); + logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL") // default to preserve earlier behavior NODE_LOCAL } @@ -71,11 +75,8 @@ private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LO /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -private[spark] class ClusterTaskSetManager( - sched: ClusterScheduler, - val taskSet: TaskSet) - extends TaskSetManager - with Logging { +private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) + extends TaskSetManager with Logging { // Maximum time to wait to run a task in a preferred location (in ms) val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong @@ -106,13 +107,14 @@ private[spark] class ClusterTaskSetManager( var runningTasks = 0 var priority = taskSet.priority var stageId = taskSet.stageId - var name = "TaskSet_"+taskSet.stageId.toString - var parent:Schedulable = null + var name = "TaskSet_" + taskSet.stageId.toString + var parent: Schedulable = null // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis - // List of pending tasks for each node (process local to container). These collections are actually + // List of pending tasks for each node (process local to container). + // These collections are actually // treated as stacks, in which new tasks are added to the end of the // ArrayBuffer and removed from the end. This makes it faster to detect // tasks that repeatedly fail because whenever a task failed, it is put @@ -172,9 +174,11 @@ private[spark] class ClusterTaskSetManager( // Note that it follows the hierarchy. // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL - private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, - taskLocality: TaskLocality.TaskLocality): HashSet[String] = { - + private def findPreferredLocations( + _taskPreferredLocations: Seq[String], + scheduler: ClusterScheduler, + taskLocality: TaskLocality.TaskLocality): HashSet[String] = + { if (TaskLocality.PROCESS_LOCAL == taskLocality) { // straight forward comparison ! Special case it. val retval = new HashSet[String]() @@ -189,13 +193,14 @@ private[spark] class ClusterTaskSetManager( return retval } - val taskPreferredLocations = + val taskPreferredLocations = { if (TaskLocality.NODE_LOCAL == taskLocality) { _taskPreferredLocations } else { assert (TaskLocality.RACK_LOCAL == taskLocality) // Expand set to include all 'seen' rack local hosts. - // This works since container allocation/management happens within master - so any rack locality information is updated in msater. + // This works since container allocation/management happens within master - + // so any rack locality information is updated in msater. // Best case effort, and maybe sort of kludge for now ... rework it later ? val hosts = new HashSet[String] _taskPreferredLocations.foreach(h => { @@ -213,6 +218,7 @@ private[spark] class ClusterTaskSetManager( hosts } + } val retval = new HashSet[String] scheduler.synchronized { @@ -229,11 +235,13 @@ private[spark] class ClusterTaskSetManager( // Add a task to all the pending-task lists that it should be on. private def addPendingTask(index: Int) { - // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate - // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL) - val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + // We can infer hostLocalLocations from rackLocalLocations by joining it against + // tasks(index).preferredLocations (with appropriate hostPort <-> host conversion). + // But not doing it for simplicity sake. If this becomes a performance issue, modify it. + val locs = tasks(index).preferredLocations + val processLocalLocations = findPreferredLocations(locs, sched, TaskLocality.PROCESS_LOCAL) + val hostLocalLocations = findPreferredLocations(locs, sched, TaskLocality.NODE_LOCAL) + val rackLocalLocations = findPreferredLocations(locs, sched, TaskLocality.RACK_LOCAL) if (rackLocalLocations.size == 0) { // Current impl ensures this. @@ -298,18 +306,24 @@ private[spark] class ClusterTaskSetManager( } // Number of pending tasks for a given host Port (which would be process local) - def numPendingTasksForHostPort(hostPort: String): Int = { - getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + override def numPendingTasksForHostPort(hostPort: String): Int = { + getPendingTasksForHostPort(hostPort).count { index => + copiesRunning(index) == 0 && !finished(index) + } } // Number of pending tasks for a given host (which would be data local) - def numPendingTasksForHost(hostPort: String): Int = { - getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + override def numPendingTasksForHost(hostPort: String): Int = { + getPendingTasksForHost(hostPort).count { index => + copiesRunning(index) == 0 && !finished(index) + } } // Number of pending rack local tasks for a given host - def numRackLocalPendingTasksForHost(hostPort: String): Int = { - getRackLocalPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + override def numRackLocalPendingTasksForHost(hostPort: String): Int = { + getRackLocalPendingTasksForHost(hostPort).count { index => + copiesRunning(index) == 0 && !finished(index) + } } @@ -337,12 +351,12 @@ private[spark] class ClusterTaskSetManager( speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set if (speculatableTasks.size > 0) { - val localTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) - } + val localTask = speculatableTasks.find { index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, + TaskLocality.NODE_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) + } if (localTask != None) { speculatableTasks -= localTask.get @@ -351,11 +365,11 @@ private[spark] class ClusterTaskSetManager( // check for rack locality if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - val rackTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - locations.contains(hostPort) && !attemptLocs.contains(hostPort) + val rackTask = speculatableTasks.find { index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, + TaskLocality.RACK_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + locations.contains(hostPort) && !attemptLocs.contains(hostPort) } if (rackTask != None) { @@ -367,7 +381,9 @@ private[spark] class ClusterTaskSetManager( // Any task ... if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { // Check for attemptLocs also ? - val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.hostPort).contains(hostPort)) + val nonLocalTask = speculatableTasks.find { i => + !taskAttempts(i).map(_.hostPort).contains(hostPort) + } if (nonLocalTask != None) { speculatableTasks -= nonLocalTask.get return nonLocalTask @@ -397,7 +413,8 @@ private[spark] class ClusterTaskSetManager( } } - // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to failed tasks later rather than sooner. + // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to + // failed tasks later rather than sooner. // TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down). val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs) if (noPrefTask != None) { @@ -433,7 +450,8 @@ private[spark] class ClusterTaskSetManager( locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined } - // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). + // Does a host count as a rack local preferred location for a task? + // (assumes host is NOT preferred location). // This is true if either the task has preferred locations and this host is one, or it has // no preferred locations (in which we still count the launch as preferred). private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { @@ -454,14 +472,22 @@ private[spark] class ClusterTaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { - + override def slaveOffer( + execId: String, + hostPort: String, + availableCpus: Double, + overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = + { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { // If explicitly specified, use that val locality = if (overrideLocality != null) overrideLocality else { // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... val time = System.currentTimeMillis - if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY + if (time - lastPreferredLaunchTime < LOCALITY_WAIT) { + TaskLocality.NODE_LOCAL + } else { + TaskLocality.ANY + } } findTask(hostPort, locality) match { @@ -504,7 +530,7 @@ private[spark] class ClusterTaskSetManager( return None } - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { state match { case TaskState.FINISHED => taskFinished(tid, state, serializedData) @@ -536,7 +562,8 @@ private[spark] class ClusterTaskSetManager( 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) + sched.listener.taskEnded( + tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) } catch { case cnf: ClassNotFoundException => val loader = Thread.currentThread().getContextClassLoader @@ -582,8 +609,8 @@ private[spark] class ClusterTaskSetManager( return case taskResultTooBig: TaskResultTooBigFailure => - logInfo("Loss was due to task %s result exceeding Akka frame size; " + - "aborting job".format(tid)) + 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)) return @@ -634,7 +661,7 @@ private[spark] class ClusterTaskSetManager( } } - def error(message: String) { + override def error(message: String) { // Save the error message abort("Error: " + message) } @@ -662,7 +689,8 @@ private[spark] class ClusterTaskSetManager( } } - //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + // TODO: for now we just find Pool not TaskSetManager, + // we can extend this function in future if needed override def getSchedulableByName(name: String): Schedulable = { return null } @@ -687,13 +715,15 @@ private[spark] class ClusterTaskSetManager( // If some task has preferred locations only on hostname, and there are no more executors there, // put it in the no-prefs list to avoid the wait from delay scheduling - // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to - // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. - // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if - // there is no host local node for the task (not if there is no process local node for the task) + // host local tasks - should we push this to rack local or no pref list ? For now, preserving + // behavior and moving to no prefs list. Note, this was done due to impliations related to + // 'waiting' for data local tasks, etc. + // Note: NOT checking process local list - since host local list is super set of that. We need + // to ad to no prefs only if there is no host local node for the task (not if there is no + // process local node for the task) for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { - // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) + val newLocs = findPreferredLocations( + tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) if (newLocs.isEmpty) { pendingTasksWithNoPrefs += index } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 07c3ddcc7e..7978a5df74 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -17,18 +17,28 @@ package spark.scheduler.cluster -import scala.collection.mutable.ArrayBuffer -import spark.scheduler._ -import spark.TaskState.TaskState import java.nio.ByteBuffer +import spark.TaskState.TaskState +import spark.scheduler.TaskSet + private[spark] trait TaskSetManager extends Schedulable { + def taskSet: TaskSet - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, - overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] + + def slaveOffer( + execId: String, + hostPort: String, + availableCpus: Double, + overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] + def numPendingTasksForHostPort(hostPort: String): Int - def numRackLocalPendingTasksForHost(hostPort :String): Int + + def numRackLocalPendingTasksForHost(hostPort: String): Int + def numPendingTasksForHost(hostPort: String): Int + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) + def error(message: String) } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 1f73cb99a7..edd83d4cb4 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -37,10 +37,15 @@ import akka.actor._ * testing fault recovery. */ -private[spark] case class LocalReviveOffers() -private[spark] case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) +private[spark] +case class LocalReviveOffers() + +private[spark] +case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) + +private[spark] +class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { -private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { def receive = { case LocalReviveOffers => launchTask(localScheduler.resourceOffer(freeCores)) @@ -55,7 +60,7 @@ private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: I freeCores -= 1 localScheduler.threadPool.submit(new Runnable { def run() { - localScheduler.runTask(task.taskId,task.serializedTask) + localScheduler.runTask(task.taskId, task.serializedTask) } }) } @@ -110,7 +115,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: override def submitTasks(taskSet: TaskSet) { synchronized { - var manager = new LocalTaskSetManager(this, taskSet) + val manager = new LocalTaskSetManager(this, taskSet) schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) activeTaskSets(taskSet.id) = manager taskSetTaskIds(taskSet.id) = new HashSet[Long]() @@ -124,14 +129,15 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val tasks = new ArrayBuffer[TaskDescription](freeCores) val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() for (manager <- sortedTaskSetQueue) { - logDebug("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) + logDebug("parentName:%s,name:%s,runningTasks:%s".format( + manager.parent.name, manager.name, manager.runningTasks)) } var launchTask = false for (manager <- sortedTaskSetQueue) { do { launchTask = false - manager.slaveOffer(null,null,freeCpuCores) match { + manager.slaveOffer(null, null, freeCpuCores) match { case Some(task) => tasks += task taskIdToTaskSetId(task.taskId) = manager.taskSet.id @@ -139,7 +145,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: freeCpuCores -= 1 launchTask = true case None => {} - } + } } while(launchTask) } return tasks diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index e662ad6709..bbce9eda64 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -17,27 +17,26 @@ package spark.scheduler.local -import java.io.File -import java.util.concurrent.atomic.AtomicInteger import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import spark._ +import spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState} import spark.TaskState.TaskState -import spark.scheduler._ -import spark.scheduler.cluster._ +import spark.scheduler.{Task, TaskResult, TaskSet} +import spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager} + + +private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) + extends TaskSetManager with Logging { -private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { var parent: Schedulable = null var weight: Int = 1 var minShare: Int = 0 var runningTasks: Int = 0 var priority: Int = taskSet.priority var stageId: Int = taskSet.stageId - var name: String = "TaskSet_"+taskSet.stageId.toString - + var name: String = "TaskSet_" + taskSet.stageId.toString var failCount = new Array[Int](taskSet.tasks.size) val taskInfos = new HashMap[Long, TaskInfo] @@ -49,49 +48,45 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val numFailures = new Array[Int](numTasks) val MAX_TASK_FAILURES = sched.maxFailures - def increaseRunningTasks(taskNum: Int): Unit = { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } + override def increaseRunningTasks(taskNum: Int): Unit = { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } } - def decreaseRunningTasks(taskNum: Int): Unit = { + override def decreaseRunningTasks(taskNum: Int): Unit = { runningTasks -= taskNum if (parent != null) { parent.decreaseRunningTasks(taskNum) } } - def addSchedulable(schedulable: Schedulable): Unit = { + override def addSchedulable(schedulable: Schedulable): Unit = { //nothing } - def removeSchedulable(schedulable: Schedulable): Unit = { + override def removeSchedulable(schedulable: Schedulable): Unit = { //nothing } - def getSchedulableByName(name: String): Schedulable = { + override def getSchedulableByName(name: String): Schedulable = { return null } - def executorLost(executorId: String, host: String): Unit = { + override def executorLost(executorId: String, host: String): Unit = { //nothing } - def checkSpeculatableTasks(): Boolean = { - return true - } + override def checkSpeculatableTasks() = true - def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] sortedTaskSetQueue += this return sortedTaskSetQueue } - def hasPendingTasks(): Boolean = { - return true - } + override def hasPendingTasks() = true def findTask(): Option[Int] = { for (i <- 0 to numTasks-1) { @@ -102,17 +97,25 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + override def slaveOffer( + execId: String, + hostPort: String, + availableCpus: Double, + overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = + { SparkEnv.set(sched.env) - logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format(availableCpus.toInt, numFinished, numTasks)) + logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format( + availableCpus.toInt, numFinished, numTasks)) if (availableCpus > 0 && numFinished < numTasks) { findTask() match { case Some(index) => val taskId = sched.attemptId.getAndIncrement() val task = taskSet.tasks(index) - val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", + TaskLocality.NODE_LOCAL) taskInfos(taskId) = info - val bytes = Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + val bytes = Task.serializeWithDependencies( + task, sched.sc.addedFiles, sched.sc.addedJars, ser) logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") val taskName = "task %s:%d".format(taskSet.id, index) copiesRunning(index) += 1 @@ -124,19 +127,19 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - def numPendingTasksForHostPort(hostPort: String): Int = { + override def numPendingTasksForHostPort(hostPort: String): Int = { return 0 } - def numRackLocalPendingTasksForHost(hostPort :String): Int = { + override def numRackLocalPendingTasksForHost(hostPort :String): Int = { return 0 } - def numPendingTasksForHost(hostPort: String): Int = { + override def numPendingTasksForHost(hostPort: String): Int = { return 0 } - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { state match { case TaskState.FINISHED => taskEnded(tid, state, serializedData) @@ -168,15 +171,18 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val task = taskSet.tasks(index) info.markFailed() decreaseRunningTasks(1) - val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) + val reason: ExceptionFailure = ser.deserialize[ExceptionFailure]( + serializedData, getClass.getClassLoader) sched.listener.taskEnded(task, reason, null, null, info, reason.metrics.getOrElse(null)) if (!finished(index)) { copiesRunning(index) -= 1 numFailures(index) += 1 val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s\n%s".format(reason.className, reason.description, locs.mkString("\n"))) + logInfo("Loss was due to %s\n%s\n%s".format( + reason.className, reason.description, locs.mkString("\n"))) if (numFailures(index) > MAX_TASK_FAILURES) { - val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(taskSet.id, index, 4, reason.description) + val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format( + taskSet.id, index, 4, reason.description) decreaseRunningTasks(runningTasks) sched.listener.taskSetFailed(taskSet, errorMessage) // need to delete failed Taskset from schedule queue @@ -185,6 +191,6 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } } - def error(message: String) { + override def error(message: String) { } } diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 8f81d0b6ee..05afcd6567 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -33,7 +33,7 @@ class DummyTaskSetManager( initNumTasks: Int, clusterScheduler: ClusterScheduler, taskSet: TaskSet) - extends ClusterTaskSetManager(clusterScheduler,taskSet) { + extends ClusterTaskSetManager(clusterScheduler, taskSet) { parent = null weight = 1 diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index f802b66cf1..a8b88d7936 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -23,21 +23,14 @@ import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import spark.LocalSparkContext - -import spark.storage.BlockManager -import spark.storage.BlockManagerId -import spark.storage.BlockManagerMaster -import spark.{Dependency, ShuffleDependency, OneToOneDependency} -import spark.FetchFailedException import spark.MapOutputTracker import spark.RDD import spark.SparkContext -import spark.SparkException import spark.Partition import spark.TaskContext -import spark.TaskEndReason - -import spark.{FetchFailed, Success} +import spark.{Dependency, ShuffleDependency, OneToOneDependency} +import spark.{FetchFailed, Success, TaskEndReason} +import spark.storage.{BlockManagerId, BlockManagerMaster} /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler From f2422d4f29abb80b9bc76c4596d1cc31d9e6d590 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:20 -0700 Subject: [PATCH 254/419] SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure. --- .../scala/spark/scheduler/DAGScheduler.scala | 29 +++++++++++++------ core/src/test/scala/spark/FailureSuite.scala | 29 ++++++++++++++----- 2 files changed, 42 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 29e879aa42..5fcd807aff 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -17,19 +17,17 @@ package spark.scheduler -import cluster.TaskInfo -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.LinkedBlockingQueue -import java.util.concurrent.TimeUnit +import java.io.NotSerializableException import java.util.Properties +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ import spark.executor.TaskMetrics -import spark.partial.ApproximateActionListener -import spark.partial.ApproximateEvaluator -import spark.partial.PartialResult +import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} +import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -258,7 +256,8 @@ class DAGScheduler( assert(partitions.size > 0) val waiter = new JobWaiter(partitions.size, resultHandler) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) + val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, + properties) return (toSubmit, waiter) } @@ -283,7 +282,7 @@ class DAGScheduler( "Total number of partitions: " + maxPartitions) } - val (toSubmit, waiter) = prepareJob( + val (toSubmit: JobSubmitted, waiter: JobWaiter[_]) = prepareJob( finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) waiter.awaitResult() match { @@ -466,6 +465,18 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") + + // Preemptively serialize the stage RDD to make sure the tasks for this stage will be + // serializable. We are catching this exception here because it would be fairly hard to + // catch the non-serializable exception down the road, where we have several different + // implementations for local scheduler and cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) + } catch { + case e: NotSerializableException => abortStage(stage, e.toString) + return + } + if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index 6c847b8fef..c3c52f9118 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -18,9 +18,6 @@ package spark import org.scalatest.FunSuite -import org.scalatest.prop.Checkers - -import scala.collection.mutable.ArrayBuffer import SparkContext._ @@ -40,7 +37,7 @@ object FailureSuiteState { } class FailureSuite extends FunSuite with LocalSparkContext { - + // Run a 3-task map job in which task 1 deterministically fails once, and check // whether the job completes successfully and we ran 4 tasks in total. test("failure in a single-stage job") { @@ -66,7 +63,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure in a two-stage job") { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map { - case (k, v) => + case (k, v) => FailureSuiteState.synchronized { FailureSuiteState.tasksRun += 1 if (k == 1 && FailureSuiteState.tasksFailed == 0) { @@ -87,15 +84,33 @@ class FailureSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => new NonSerializable) - val thrown = intercept[spark.SparkException] { + val thrown = intercept[SparkException] { results.collect() } - assert(thrown.getClass === classOf[spark.SparkException]) + assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) FailureSuiteState.clear() } + test("failure because task closure is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + val thrown = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => a).count() + } + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("NotSerializableException")) + + val thrown1 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() + } + assert(thrown1.getClass === classOf[SparkException]) + assert(thrown1.getMessage.contains("NotSerializableException")) + + FailureSuiteState.clear() + } + // TODO: Need to add tests with shuffle fetch failures. } From 383684daaa62bebc177b4c74573ce509f154f74b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:33:27 -0700 Subject: [PATCH 255/419] Replaces Seq with HashSet, removes redundant import --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index bb2b003486..606e1eb2fc 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -5,7 +5,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Properties import spark.{ExceptionFailure, Logging, SparkContext, Success, Utils} @@ -18,7 +18,6 @@ import spark.ui.JettyUtils._ import spark.ui.Page.Executors import spark.ui.UIUtils.headerSparkPage import spark.ui.UIUtils -import spark.Utils import scala.xml.{Node, XML} @@ -114,7 +113,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, Seq[Long]]() + val executorToTasksActive = HashMap[String, HashSet[Long]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = @@ -122,12 +121,14 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()) :+ taskStart.taskInfo.taskId + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) + + taskStart.taskInfo.taskId } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()).filterNot(_ == taskEnd.taskInfo.taskId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) - + taskEnd.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => From abc78cd3318fb7bc69d10fd5422d20b299a8d7d8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:47:16 -0700 Subject: [PATCH 256/419] Modifies instead of copies HashSets, fixes comment style --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index b02bf8f4bf..7bf50de660 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -52,7 +52,7 @@ class DAGScheduler( } taskSched.setListener(this) - //Called by TaskScheduler to report task's starting. + // Called by TaskScheduler to report task's starting. override def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventQueue.put(BeginEvent(task, taskInfo)) } diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 606e1eb2fc..69fb306074 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -121,14 +121,16 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) + - taskStart.taskInfo.taskId + if (!executorToTasksActive.contains(eid)) + executorToTasksActive(eid) = HashSet[Long]() + executorToTasksActive(eid) += taskStart.taskInfo.taskId } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) - - taskEnd.taskInfo.taskId + if (!executorToTasksActive.contains(eid)) + executorToTasksActive(eid) = HashSet[Long]() + executorToTasksActive(eid) -= taskStart.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => From 6a31b7191d5a8203563a2a4e600210c67439abf5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 23 Jul 2013 16:20:24 -0700 Subject: [PATCH 257/419] Small bug fix --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 69fb306074..bad5c442ab 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -130,7 +130,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val eid = taskEnd.taskInfo.executorId if (!executorToTasksActive.contains(eid)) executorToTasksActive(eid) = HashSet[Long]() - executorToTasksActive(eid) -= taskStart.taskInfo.taskId + executorToTasksActive(eid) -= taskEnd.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -146,4 +146,4 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { executorToTaskInfos(eid) = taskList } } -} \ No newline at end of file +} From 85ab8114bc1367a0f4f32d4b8635c41aa547bc72 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:25:58 -0700 Subject: [PATCH 258/419] Moved non-serializable closure catching exception from submitStage to submitMissingTasks --- .../scala/spark/scheduler/DAGScheduler.scala | 25 ++++++++++--------- core/src/test/scala/spark/FailureSuite.scala | 10 ++++++++ 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 5fcd807aff..fde998494f 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -465,18 +465,6 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") - - // Preemptively serialize the stage RDD to make sure the tasks for this stage will be - // serializable. We are catching this exception here because it would be fairly hard to - // catch the non-serializable exception down the road, where we have several different - // implementations for local scheduler and cluster schedulers. - try { - SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) - } catch { - case e: NotSerializableException => abortStage(stage, e.toString) - return - } - if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) @@ -515,6 +503,19 @@ class DAGScheduler( } } if (tasks.size > 0) { + // Preemptively serialize a task to make sure it can be serialized. We are catching this + // exception here because it would be fairly hard to catch the non-serializable exception + // down the road, where we have several different implementations for local scheduler and + // cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) + } catch { + case e: NotSerializableException => + abortStage(stage, e.toString) + running -= stage + return + } + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index c3c52f9118..5b133cdd6e 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -96,18 +96,28 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure because task closure is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable + + // Non-serializable closure in the final result stage val thrown = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => a).count() } assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) + // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() } assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) + // Non-serializable closure in foreach function + val thrown2 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).foreach(x => println(a)) + } + assert(thrown2.getClass === classOf[SparkException]) + assert(thrown2.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() } From d33b8a2a0ffd6d085cbd8de22863a1f35c106270 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:28:39 -0700 Subject: [PATCH 259/419] Added comments on task closure serialization. --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 ++ .../main/scala/spark/scheduler/local/LocalTaskSetManager.scala | 2 ++ 2 files changed, 4 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index d934293b70..f64818876b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -515,6 +515,8 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: } // Serialize and return the task val startTime = System.currentTimeMillis + // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here + // we assume the task can be serialized without exceptions. val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) val timeTaken = System.currentTimeMillis - startTime diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index bbce9eda64..a9b49cad0e 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -114,6 +114,8 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) taskInfos(taskId) = info + // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here + // we assume the task can be serialized without exceptions. val bytes = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") From 3dae1df66f34d3716c697f85cfe4aedeee428688 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:25:58 -0700 Subject: [PATCH 260/419] Moved non-serializable closure catching exception from submitStage to submitMissingTasks --- .../scala/spark/scheduler/DAGScheduler.scala | 25 ++++++++++--------- core/src/test/scala/spark/FailureSuite.scala | 10 ++++++++ 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 5fcd807aff..fde998494f 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -465,18 +465,6 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") - - // Preemptively serialize the stage RDD to make sure the tasks for this stage will be - // serializable. We are catching this exception here because it would be fairly hard to - // catch the non-serializable exception down the road, where we have several different - // implementations for local scheduler and cluster schedulers. - try { - SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) - } catch { - case e: NotSerializableException => abortStage(stage, e.toString) - return - } - if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) @@ -515,6 +503,19 @@ class DAGScheduler( } } if (tasks.size > 0) { + // Preemptively serialize a task to make sure it can be serialized. We are catching this + // exception here because it would be fairly hard to catch the non-serializable exception + // down the road, where we have several different implementations for local scheduler and + // cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) + } catch { + case e: NotSerializableException => + abortStage(stage, e.toString) + running -= stage + return + } + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index c3c52f9118..5b133cdd6e 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -96,18 +96,28 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure because task closure is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable + + // Non-serializable closure in the final result stage val thrown = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => a).count() } assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) + // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() } assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) + // Non-serializable closure in foreach function + val thrown2 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).foreach(x => println(a)) + } + assert(thrown2.getClass === classOf[SparkException]) + assert(thrown2.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() } From 503acd3a379a3686d343fdf072fc231b8fba78f9 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:47:27 +0800 Subject: [PATCH 261/419] Build metrics system framwork --- conf/metrics.properties | 13 ++++ .../metrics/AbstractInstrumentation.scala | 68 +++++++++++++++++++ .../scala/spark/metrics/MetricsConfig.scala | 55 +++++++++++++++ .../spark/metrics/sink/ConsoleSink.scala | 43 ++++++++++++ .../scala/spark/metrics/sink/CsvSink.scala | 53 +++++++++++++++ .../scala/spark/metrics/sink/JmxSink.scala | 17 +++++ .../main/scala/spark/metrics/sink/Sink.scala | 6 ++ 7 files changed, 255 insertions(+) create mode 100644 conf/metrics.properties create mode 100644 core/src/main/scala/spark/metrics/AbstractInstrumentation.scala create mode 100644 core/src/main/scala/spark/metrics/MetricsConfig.scala create mode 100644 core/src/main/scala/spark/metrics/sink/ConsoleSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/CsvSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/JmxSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/Sink.scala diff --git a/conf/metrics.properties b/conf/metrics.properties new file mode 100644 index 0000000000..78749cf381 --- /dev/null +++ b/conf/metrics.properties @@ -0,0 +1,13 @@ +# syntax: [prefix].[sink].[instance].[options] + +*.sink.console.period=10 + +*.sink.console.unit=second + +master.sink.console.period=10 + +master.sink.console.unit=second + +worker.sink.console.period=20 + +worker.sink.console.unit=second diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala new file mode 100644 index 0000000000..0fed608488 --- /dev/null +++ b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala @@ -0,0 +1,68 @@ +package spark.metrics + +import scala.collection.mutable + +import com.codahale.metrics.{JmxReporter, MetricRegistry} + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import spark.Logging +import spark.metrics.sink._ + +trait AbstractInstrumentation extends Logging { + initLogging() + + def registryHandler: MetricRegistry + def instance: String + + val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) + val metricsConfig = new MetricsConfig(confFile) + + val sinks = new mutable.ArrayBuffer[Sink] + + def registerSinks() { + val instConfig = metricsConfig.getInstance(instance) + val sinkConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SINK_REGEX) + + // Register JMX sink as a default sink + sinks += new JmxSink(registryHandler) + + // Register other sinks according to conf + sinkConfigs.foreach { kv => + val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { + AbstractInstrumentation.DEFAULT_SINKS(kv._1) + } else { + kv._2.getProperty("class") + } + try { + val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) + .newInstance(kv._2, registryHandler) + sinks += sink.asInstanceOf[Sink] + } catch { + case e: Exception => logError("class " + classPath + "cannot be instantialize", e) + } + } + + sinks.foreach(_.registerSink) + } + + def unregisterSinks() { + sinks.foreach(_.unregisterSink) + } +} + +object AbstractInstrumentation { + val DEFAULT_SINKS = Map( + "console" -> "spark.metrics.sink.ConsoleSink", + "csv" -> "spark.metrics.sink.CsvSink") + + val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + + val timeUnits = Map( + "millisecond" -> TimeUnit.MILLISECONDS, + "second" -> TimeUnit.SECONDS, + "minute" -> TimeUnit.MINUTES, + "hour" -> TimeUnit.HOURS, + "day" -> TimeUnit.DAYS) +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala new file mode 100644 index 0000000000..0fec1988ea --- /dev/null +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -0,0 +1,55 @@ +package spark.metrics + +import java.util.Properties +import java.io.FileInputStream + +import scala.collection.mutable +import scala.util.matching.Regex + +class MetricsConfig(val configFile: String) { + val properties = new Properties() + var fis: FileInputStream = _ + + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } + + val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) + if (propertyCategories.contains(MetricsConfig.DEFAULT_PREFIX)) { + import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(MetricsConfig.DEFAULT_PREFIX) + for ((inst, prop) <- propertyCategories; p <- defaultProperty + if inst != MetricsConfig.DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { + prop.setProperty(p._1, p._2) + } + } + + def getInstance(inst: String) = { + propertyCategories.get(inst) match { + case Some(s) => s + case None => propertyCategories(MetricsConfig.DEFAULT_PREFIX) + } + } +} + +object MetricsConfig { + val DEFAULT_CONFIG_FILE = "/home/jerryshao/project/sotc_cloud-spark/conf/metrics.properties" + val DEFAULT_PREFIX = "*" + val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + + def subProperties(prop: Properties, regex: Regex) = { + val subProperties = new mutable.HashMap[String, Properties] + + import scala.collection.JavaConversions._ + prop.foreach { kv => + val regex(a, b) = kv._1 + subProperties.getOrElseUpdate(a, new Properties).setProperty(b, kv._2) + println(">>>>>subProperties added " + a + " " + b + " " + kv._2) + } + + subProperties + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala new file mode 100644 index 0000000000..5426af8c4c --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -0,0 +1,43 @@ +package spark.metrics.sink + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => ConsoleSink.CONSOLE_DEFAULT_PERIOD.toInt + } + + val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { + case Some(s) => AbstractInstrumentation.timeUnits(s) + case None => AbstractInstrumentation.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + } + + var reporter: ConsoleReporter = _ + + override def registerSink() { + reporter = ConsoleReporter.forRegistry(registry) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build() + + reporter.start(pollPeriod, pollUnit) + } + + override def unregisterSink() { + reporter.stop() + } +} + +object ConsoleSink { + val CONSOLE_DEFAULT_PERIOD = "10" + val CONSOLE_DEFAULT_UNIT = "second" + + val CONSOLE_KEY_PERIOD = "period" + val CONSOLE_KEY_UNIT = "unit" +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala new file mode 100644 index 0000000000..3a80c36901 --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -0,0 +1,53 @@ +package spark.metrics.sink + +import java.io.File +import java.util.{Locale, Properties} +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.{CsvReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => CsvSink.CSV_DEFAULT_PERIOD.toInt + } + + val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { + case Some(s) => AbstractInstrumentation.timeUnits(s) + case None => AbstractInstrumentation.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + } + + val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { + case Some(s) => s + case None => CsvSink.CSV_DEFAULT_DIR + } + + var reporter: CsvReporter = _ + + override def registerSink() { + reporter = CsvReporter.forRegistry(registry) + .formatFor(Locale.US) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build(new File(pollDir)) + + reporter.start(pollPeriod, pollUnit) + } + + override def unregisterSink() { + reporter.stop() + } +} + +object CsvSink { + val CSV_KEY_PERIOD = "period" + val CSV_KEY_UNIT = "unit" + val CSV_KEY_DIR = "directory" + + val CSV_DEFAULT_PERIOD = "1" + val CSV_DEFAULT_UNIT = "minute" + val CSV_DEFAULT_DIR = "/tmp/" +} + diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala new file mode 100644 index 0000000000..56e5677700 --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -0,0 +1,17 @@ +package spark.metrics.sink + +import com.codahale.metrics.{JmxReporter, MetricRegistry} + +class JmxSink(registry: MetricRegistry) extends Sink { + var reporter: JmxReporter = _ + + override def registerSink() { + reporter = JmxReporter.forRegistry(registry).build() + reporter.start() + } + + override def unregisterSink() { + reporter.stop() + } + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala new file mode 100644 index 0000000000..65ebcb4eac --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -0,0 +1,6 @@ +package spark.metrics.sink + +trait Sink { + def registerSink: Unit + def unregisterSink: Unit +} \ No newline at end of file From 9dec8c73e6f0c3b6b55a11ff92cc9bff18dadd24 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:48:41 +0800 Subject: [PATCH 262/419] Add Master and Worker instrumentation support --- .../scala/spark/deploy/master/Master.scala | 9 ++- .../deploy/master/MasterInstrumentation.scala | 44 +++++++++++++++ .../scala/spark/deploy/worker/Worker.scala | 11 +++- .../deploy/worker/WorkerInstrumentation.scala | 55 +++++++++++++++++++ 4 files changed, 115 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala create mode 100644 core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index eddcafd84d..3a7c4e5a52 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -33,7 +33,8 @@ import spark.util.AkkaUtils import ui.MasterWebUI -private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { +private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor +with Logging with MasterInstrumentation { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 @@ -73,6 +74,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) + + initialize(this) } override def postStop() { @@ -316,6 +319,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } + + override def postStop() { + uninitialize() + } } private[spark] object Master { diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala new file mode 100644 index 0000000000..13088189a4 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -0,0 +1,44 @@ +package spark.deploy.master + +import com.codahale.metrics.{Gauge, JmxReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +private[spark] trait MasterInstrumentation extends AbstractInstrumentation { + var masterInst: Option[Master] = None + val metricRegistry = new MetricRegistry() + + override def registryHandler = metricRegistry + + override def instance = "master" + + def initialize(master: Master) { + masterInst = Some(master) + + // Register and start all the sinks + registerSinks + } + + def uninitialize() { + unregisterSinks + } + + // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "workers", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.workers.size).getOrElse(0) + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "apps", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.apps.size).getOrElse(0) + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "waiting_apps", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.waitingApps.size).getOrElse(0) + }) + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 0bd88ea253..b64bdb8d28 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -41,7 +41,7 @@ private[spark] class Worker( memory: Int, masterUrl: String, workDirPath: String = null) - extends Actor with Logging { + extends Actor with Logging with WorkerInstrumentation { Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -97,6 +97,9 @@ private[spark] class Worker( webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() + startWebUi() + + initialize(this) } def connectToMaster() { @@ -155,10 +158,10 @@ private[spark] class Worker( case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => masterDisconnected() - + case RequestWorkerState => { sender ! WorkerState(host, port, workerId, executors.values.toList, - finishedExecutors.values.toList, masterUrl, cores, memory, + finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed, masterWebUiUrl) } } @@ -178,6 +181,8 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) webUi.stop() + + uninitialize() } } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala new file mode 100644 index 0000000000..04c43ce33b --- /dev/null +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -0,0 +1,55 @@ +package spark.deploy.worker + +import com.codahale.metrics.{JmxReporter, Gauge, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { + var workerInst: Option[Worker] = None + val metricRegistry = new MetricRegistry() + + override def registryHandler = metricRegistry + + override def instance = "worker" + + def initialize(worker: Worker) { + workerInst = Some(worker) + + // Register and start all the sinks + registerSinks() + } + + def uninitialize() { + unregisterSinks() + } + + // Gauge for executors number + metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) + }) + + // Gauge for cores used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "Mbytes"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) + }) + + // Gauge for cores free of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) + }) +} \ No newline at end of file From c3daad3f65630eb4ed536d06c0d467cde57a8142 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 12:00:19 +0800 Subject: [PATCH 263/419] Update metric source support for instrumentation --- conf/metrics.properties | 4 ++- .../deploy/master/MasterInstrumentation.scala | 7 +++-- .../deploy/worker/WorkerInstrumentation.scala | 5 +++- .../metrics/AbstractInstrumentation.scala | 29 +++++++++++++++++-- .../scala/spark/metrics/MetricsConfig.scala | 12 ++++---- .../scala/spark/metrics/sink/CsvSink.scala | 4 +-- .../main/scala/spark/metrics/sink/Sink.scala | 1 + .../spark/metrics/source/JvmSource.scala | 17 +++++++++++ .../scala/spark/metrics/source/Source.scala | 5 ++++ 9 files changed, 70 insertions(+), 14 deletions(-) create mode 100644 core/src/main/scala/spark/metrics/source/JvmSource.scala create mode 100644 core/src/main/scala/spark/metrics/source/Source.scala diff --git a/conf/metrics.properties b/conf/metrics.properties index 78749cf381..0bbb6b5229 100644 --- a/conf/metrics.properties +++ b/conf/metrics.properties @@ -1,9 +1,11 @@ -# syntax: [prefix].[sink].[instance].[options] +# syntax: [prefix].[sink|source].[instance].[options] *.sink.console.period=10 *.sink.console.unit=second +master.source.jvm.class=spark.metrics.source.JvmSource + master.sink.console.period=10 master.sink.console.unit=second diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 13088189a4..c295e725d7 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -15,12 +15,15 @@ private[spark] trait MasterInstrumentation extends AbstractInstrumentation { def initialize(master: Master) { masterInst = Some(master) + // Register all the sources + registerSources() + // Register and start all the sinks - registerSinks + registerSinks() } def uninitialize() { - unregisterSinks + unregisterSinks() } // Gauge for worker numbers in cluster diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 04c43ce33b..2f725300b5 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -15,6 +15,9 @@ private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { def initialize(worker: Worker) { workerInst = Some(worker) + // Register all the sources + registerSources() + // Register and start all the sinks registerSinks() } @@ -36,7 +39,7 @@ private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "Mbytes"), + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), new Gauge[Int] { override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) }) diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala index 0fed608488..9cae1e0220 100644 --- a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala +++ b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala @@ -9,17 +9,39 @@ import java.util.concurrent.TimeUnit import spark.Logging import spark.metrics.sink._ +import spark.metrics.source._ -trait AbstractInstrumentation extends Logging { +private [spark] trait AbstractInstrumentation extends Logging { initLogging() + // Get MetricRegistry handler def registryHandler: MetricRegistry + // Get the instance name def instance: String val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] + val sources = new mutable.ArrayBuffer[Source] + + def registerSources() { + val instConfig = metricsConfig.getInstance(instance) + val sourceConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SOURCE_REGEX) + + // Register all the sources + sourceConfigs.foreach { kv => + val classPath = kv._2.getProperty("class") + try { + val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) + .newInstance(registryHandler) + sources += source.asInstanceOf[Source] + } catch { + case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) + } + } + sources.foreach(_.registerSource) + } def registerSinks() { val instConfig = metricsConfig.getInstance(instance) @@ -33,6 +55,7 @@ trait AbstractInstrumentation extends Logging { val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { AbstractInstrumentation.DEFAULT_SINKS(kv._1) } else { + // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") } try { @@ -40,10 +63,9 @@ trait AbstractInstrumentation extends Logging { .newInstance(kv._2, registryHandler) sinks += sink.asInstanceOf[Sink] } catch { - case e: Exception => logError("class " + classPath + "cannot be instantialize", e) + case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.registerSink) } @@ -58,6 +80,7 @@ object AbstractInstrumentation { "csv" -> "spark.metrics.sink.CsvSink") val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r val timeUnits = Map( "millisecond" -> TimeUnit.MILLISECONDS, diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 0fec1988ea..be4f670918 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -6,7 +6,7 @@ import java.io.FileInputStream import scala.collection.mutable import scala.util.matching.Regex -class MetricsConfig(val configFile: String) { +private [spark] class MetricsConfig(val configFile: String) { val properties = new Properties() var fis: FileInputStream = _ @@ -36,7 +36,7 @@ class MetricsConfig(val configFile: String) { } object MetricsConfig { - val DEFAULT_CONFIG_FILE = "/home/jerryshao/project/sotc_cloud-spark/conf/metrics.properties" + val DEFAULT_CONFIG_FILE = "conf/metrics.properties" val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r @@ -45,9 +45,11 @@ object MetricsConfig { import scala.collection.JavaConversions._ prop.foreach { kv => - val regex(a, b) = kv._1 - subProperties.getOrElseUpdate(a, new Properties).setProperty(b, kv._2) - println(">>>>>subProperties added " + a + " " + b + " " + kv._2) + if (regex.findPrefixOf(kv._1) != None) { + val regex(prefix, suffix) = kv._1 + subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) + println(">>>>>subProperties added " + prefix + " " + suffix + " " + kv._2) + } } subProperties diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 3a80c36901..1d663f6cff 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -46,8 +46,8 @@ object CsvSink { val CSV_KEY_UNIT = "unit" val CSV_KEY_DIR = "directory" - val CSV_DEFAULT_PERIOD = "1" - val CSV_DEFAULT_UNIT = "minute" + val CSV_DEFAULT_PERIOD = "10" + val CSV_DEFAULT_UNIT = "second" val CSV_DEFAULT_DIR = "/tmp/" } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 65ebcb4eac..26052b7231 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -2,5 +2,6 @@ package spark.metrics.sink trait Sink { def registerSink: Unit + def unregisterSink: Unit } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala new file mode 100644 index 0000000000..8f6bf48843 --- /dev/null +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -0,0 +1,17 @@ +package spark.metrics.source + +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} + +class JvmSource(registry: MetricRegistry) extends Source { + // Initialize memory usage gauge for jvm + val memUsageMetricSet = new MemoryUsageGaugeSet + + // Initialize garbage collection usage gauge for jvm + val gcMetricSet = new GarbageCollectorMetricSet + + override def registerSource() { + registry.registerAll(memUsageMetricSet) + registry.registerAll(gcMetricSet) + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala new file mode 100644 index 0000000000..35cfe0c8ff --- /dev/null +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -0,0 +1,5 @@ +package spark.metrics.source + +trait Source { + def registerSource: Unit +} \ No newline at end of file From 03f9871116801abbdd7b4c7892c8d6affb1c4d9e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 18:29:54 +0800 Subject: [PATCH 264/419] MetricsSystem refactor --- .../scala/spark/deploy/master/Master.scala | 17 ++- .../deploy/master/MasterInstrumentation.scala | 66 ++++----- .../scala/spark/deploy/worker/Worker.scala | 12 +- .../deploy/worker/WorkerInstrumentation.scala | 131 +++++++++++------- ...trumentation.scala => MetricsSystem.scala} | 54 +++++--- .../spark/metrics/sink/ConsoleSink.scala | 10 +- .../scala/spark/metrics/sink/CsvSink.scala | 10 +- .../scala/spark/metrics/sink/JmxSink.scala | 4 +- .../main/scala/spark/metrics/sink/Sink.scala | 4 +- .../spark/metrics/source/JvmSource.scala | 30 ++-- .../scala/spark/metrics/source/Source.scala | 6 +- 11 files changed, 194 insertions(+), 150 deletions(-) rename core/src/main/scala/spark/metrics/{AbstractInstrumentation.scala => MetricsSystem.scala} (64%) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3a7c4e5a52..e44f5e3168 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -29,12 +29,12 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import spark.deploy._ import spark.{Logging, SparkException, Utils} +import spark.metrics.MetricsSystem import spark.util.AkkaUtils import ui.MasterWebUI -private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor -with Logging with MasterInstrumentation { +private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 @@ -57,6 +57,8 @@ with Logging with MasterInstrumentation { val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") + + val masterInstrumentation = new MasterInstrumentation(this) val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -75,7 +77,7 @@ with Logging with MasterInstrumentation { webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - initialize(this) + Master.metricsSystem.registerSource(masterInstrumentation) } override def postStop() { @@ -319,21 +321,22 @@ with Logging with MasterInstrumentation { removeWorker(worker) } } - - override def postStop() { - uninitialize() - } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r + + private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) + + metricsSystem.start() actorSystem.awaitTermination() + metricsSystem.stop() } /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index c295e725d7..5ea9a90319 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -1,47 +1,35 @@ package spark.deploy.master -import com.codahale.metrics.{Gauge, JmxReporter, MetricRegistry} +import java.util.{Map, HashMap => JHashMap} -import spark.metrics.AbstractInstrumentation +import com.codahale.metrics.{Gauge, Metric} -private[spark] trait MasterInstrumentation extends AbstractInstrumentation { - var masterInst: Option[Master] = None - val metricRegistry = new MetricRegistry() - - override def registryHandler = metricRegistry - - override def instance = "master" - - def initialize(master: Master) { - masterInst = Some(master) +import spark.metrics.source.Source + +private[spark] class MasterInstrumentation(val master: Master) extends Source { + val className = classOf[Master].getName() + val instrumentationName = "master" - // Register all the sources - registerSources() + override def sourceName = instrumentationName + + override def getMetrics(): Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] - // Register and start all the sinks - registerSinks() + // Gauge for worker numbers in cluster + gauges.put(className + ".workers.number", new Gauge[Int] { + override def getValue: Int = master.workers.size + }) + + // Gauge for application numbers in cluster + gauges.put(className + ".apps.number", new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + gauges.put(className + ".waiting_apps.number", new Gauge[Int] { + override def getValue: Int = master.waitingApps.size + }) + + gauges } - - def uninitialize() { - unregisterSinks() - } - - // Gauge for worker numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "workers", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.workers.size).getOrElse(0) - }) - - // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "apps", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.apps.size).getOrElse(0) - }) - - // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "waiting_apps", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.waitingApps.size).getOrElse(0) - }) - } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index b64bdb8d28..eaa1c1806f 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -23,6 +23,7 @@ import akka.util.duration._ import spark.{Logging, Utils} import spark.util.AkkaUtils import spark.deploy._ +import spark.metrics.MetricsSystem import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.text.SimpleDateFormat import java.util.Date @@ -41,7 +42,7 @@ private[spark] class Worker( memory: Int, masterUrl: String, workDirPath: String = null) - extends Actor with Logging with WorkerInstrumentation { + extends Actor with Logging { Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -67,6 +68,8 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + val workerInstrumentation = new WorkerInstrumentation(this) + def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -99,7 +102,8 @@ private[spark] class Worker( connectToMaster() startWebUi() - initialize(this) + Worker.metricsSystem.registerSource(workerInstrumentation) + Worker.metricsSystem.start() } def connectToMaster() { @@ -182,11 +186,13 @@ private[spark] class Worker( executors.values.foreach(_.kill()) webUi.stop() - uninitialize() + Worker.metricsSystem.stop() } } private[spark] object Worker { + private val metricsSystem = MetricsSystem.createMetricsSystem("worker") + def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 2f725300b5..37fd154859 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -1,58 +1,89 @@ package spark.deploy.worker -import com.codahale.metrics.{JmxReporter, Gauge, MetricRegistry} +import com.codahale.metrics.{Gauge, Metric} -import spark.metrics.AbstractInstrumentation +import java.util.{Map, HashMap => JHashMap} -private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { - var workerInst: Option[Worker] = None - val metricRegistry = new MetricRegistry() +import spark.metrics.source.Source + +private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { + val className = classOf[Worker].getName() - override def registryHandler = metricRegistry - - override def instance = "worker" - - def initialize(worker: Worker) { - workerInst = Some(worker) + override def sourceName = "worker" - // Register all the sources - registerSources() + override def getMetrics: Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] - // Register and start all the sinks - registerSinks() + // Gauge for executors number + gauges.put(className + ".executor.number", new Gauge[Int]{ + override def getValue: Int = worker.executors.size + }) + + gauges.put(className + ".core_used.number", new Gauge[Int]{ + override def getValue: Int = worker.coresUsed + }) + + gauges.put(className + ".mem_used.MBytes", new Gauge[Int]{ + override def getValue: Int = worker.memoryUsed + }) + + gauges.put(className + ".core_free.number", new Gauge[Int]{ + override def getValue: Int = worker.coresFree + }) + + gauges.put(className + ".mem_free.MBytes", new Gauge[Int]{ + override def getValue: Int = worker.memoryFree + }) + + gauges } - - def uninitialize() { - unregisterSinks() - } - - // Gauge for executors number - metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) - }) - - // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) - }) - - // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) - }) -} \ No newline at end of file +} +//private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { +// var workerInst: Option[Worker] = None +// val metricRegistry = new MetricRegistry() +// +// override def registryHandler = metricRegistry +// +// override def instance = "worker" +// +// def initialize(worker: Worker) { +// workerInst = Some(worker) +// +// registerSources() +// registerSinks() +// } +// +// def uninitialize() { +// unregisterSinks() +// } +// +// // Gauge for executors number +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) +// }) +// +// // Gauge for cores used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) +// }) +// +// // Gauge for memory used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) +// }) +// +// // Gauge for cores free of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) +// }) +// +// // Gauge for memory used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) +// }) +//} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala similarity index 64% rename from core/src/main/scala/spark/metrics/AbstractInstrumentation.scala rename to core/src/main/scala/spark/metrics/MetricsSystem.scala index 9cae1e0220..ea1bc490b5 100644 --- a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -2,7 +2,7 @@ package spark.metrics import scala.collection.mutable -import com.codahale.metrics.{JmxReporter, MetricRegistry} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit @@ -11,70 +11,76 @@ import spark.Logging import spark.metrics.sink._ import spark.metrics.source._ -private [spark] trait AbstractInstrumentation extends Logging { +private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - // Get MetricRegistry handler - def registryHandler: MetricRegistry - // Get the instance name - def instance: String - val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] + def start() { + registerSources() + registerSinks() + } + + def stop() { + sinks.foreach(_.stop) + } + + def registerSource(source: Source) { + sources += source + MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + } + def registerSources() { val instConfig = metricsConfig.getInstance(instance) - val sourceConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SOURCE_REGEX) + val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - // Register all the sources + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { - val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) - .newInstance(registryHandler) + val source = Class.forName(classPath).newInstance() sources += source.asInstanceOf[Source] + MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) } } - sources.foreach(_.registerSource) } def registerSinks() { val instConfig = metricsConfig.getInstance(instance) - val sinkConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SINK_REGEX) + val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) // Register JMX sink as a default sink - sinks += new JmxSink(registryHandler) + sinks += new JmxSink(MetricsSystem.registry) // Register other sinks according to conf sinkConfigs.foreach { kv => - val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { - AbstractInstrumentation.DEFAULT_SINKS(kv._1) + val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { + MetricsSystem.DEFAULT_SINKS(kv._1) } else { // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") } try { val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) - .newInstance(kv._2, registryHandler) + .newInstance(kv._2, MetricsSystem.registry) sinks += sink.asInstanceOf[Sink] } catch { case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.registerSink) - } - - def unregisterSinks() { - sinks.foreach(_.unregisterSink) + sinks.foreach(_.start) } } -object AbstractInstrumentation { +private[spark] object MetricsSystem { + val registry = new MetricRegistry() + val DEFAULT_SINKS = Map( "console" -> "spark.metrics.sink.ConsoleSink", "csv" -> "spark.metrics.sink.CsvSink") @@ -88,4 +94,6 @@ object AbstractInstrumentation { "minute" -> TimeUnit.MINUTES, "hour" -> TimeUnit.HOURS, "day" -> TimeUnit.DAYS) + + def createMetricsSystem(instance: String) = new MetricsSystem(instance) } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 5426af8c4c..b49a211fb3 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -5,7 +5,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{ConsoleReporter, MetricRegistry} -import spark.metrics.AbstractInstrumentation +import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { @@ -14,13 +14,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend } val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { - case Some(s) => AbstractInstrumentation.timeUnits(s) - case None => AbstractInstrumentation.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + case Some(s) => MetricsSystem.timeUnits(s) + case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } var reporter: ConsoleReporter = _ - override def registerSink() { + override def start() { reporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) @@ -29,7 +29,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend reporter.start(pollPeriod, pollUnit) } - override def unregisterSink() { + override def stop() { reporter.stop() } } diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 1d663f6cff..3f572c8e05 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -6,7 +6,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{CsvReporter, MetricRegistry} -import spark.metrics.AbstractInstrumentation +import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { @@ -15,8 +15,8 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si } val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { - case Some(s) => AbstractInstrumentation.timeUnits(s) - case None => AbstractInstrumentation.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + case Some(s) => MetricsSystem.timeUnits(s) + case None => MetricsSystem.timeUnits(CsvSink.CSV_DEFAULT_UNIT) } val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { @@ -26,7 +26,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si var reporter: CsvReporter = _ - override def registerSink() { + override def start() { reporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) @@ -36,7 +36,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si reporter.start(pollPeriod, pollUnit) } - override def unregisterSink() { + override def stop() { reporter.stop() } } diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index 56e5677700..e223dc26e9 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -5,12 +5,12 @@ import com.codahale.metrics.{JmxReporter, MetricRegistry} class JmxSink(registry: MetricRegistry) extends Sink { var reporter: JmxReporter = _ - override def registerSink() { + override def start() { reporter = JmxReporter.forRegistry(registry).build() reporter.start() } - override def unregisterSink() { + override def stop() { reporter.stop() } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 26052b7231..9fef894fde 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -1,7 +1,7 @@ package spark.metrics.sink trait Sink { - def registerSink: Unit + def start: Unit - def unregisterSink: Unit + def stop: Unit } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 8f6bf48843..7a7c1b6ffb 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -1,17 +1,23 @@ package spark.metrics.source -import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} +import java.util.{Map, HashMap => JHashMap} -class JvmSource(registry: MetricRegistry) extends Source { - // Initialize memory usage gauge for jvm - val memUsageMetricSet = new MemoryUsageGaugeSet - - // Initialize garbage collection usage gauge for jvm - val gcMetricSet = new GarbageCollectorMetricSet - - override def registerSource() { - registry.registerAll(memUsageMetricSet) - registry.registerAll(gcMetricSet) +import com.codahale.metrics.Metric +import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} + +class JvmSource extends Source { + override def sourceName = "jvm" + + override def getMetrics(): Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] + + import scala.collection.JavaConversions._ + val gcMetricSet = new GarbageCollectorMetricSet + gcMetricSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) + + val memGaugeSet = new MemoryUsageGaugeSet + memGaugeSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) + + gauges } } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index 35cfe0c8ff..edd59de46a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,5 +1,7 @@ package spark.metrics.source -trait Source { - def registerSource: Unit +import com.codahale.metrics.MetricSet + +trait Source extends MetricSet { + def sourceName: String } \ No newline at end of file From 4d6dd67fa1f8f031f1ef46a442cec733fa3b1a7a Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 28 Jun 2013 02:26:55 +0800 Subject: [PATCH 265/419] refactor metrics system 1.change source abstract class to support MetricRegistry 2.change master/work/jvm source class --- .../deploy/master/MasterInstrumentation.scala | 44 ++++----- .../deploy/worker/WorkerInstrumentation.scala | 98 +++++-------------- .../scala/spark/metrics/MetricsSystem.scala | 23 ++--- .../spark/metrics/sink/ConsoleSink.scala | 11 +-- .../scala/spark/metrics/sink/CsvSink.scala | 9 +- .../scala/spark/metrics/sink/JmxSink.scala | 5 +- .../spark/metrics/source/JvmSource.scala | 22 ++--- .../scala/spark/metrics/source/Source.scala | 6 +- 8 files changed, 84 insertions(+), 134 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 5ea9a90319..46c90b94d2 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -4,32 +4,32 @@ import java.util.{Map, HashMap => JHashMap} import com.codahale.metrics.{Gauge, Metric} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} + import spark.metrics.source.Source +import spark.Logging private[spark] class MasterInstrumentation(val master: Master) extends Source { val className = classOf[Master].getName() val instrumentationName = "master" - - override def sourceName = instrumentationName - - override def getMetrics(): Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - // Gauge for worker numbers in cluster - gauges.put(className + ".workers.number", new Gauge[Int] { + val metricRegistry = new MetricRegistry() + val sourceName = instrumentationName + + metricRegistry.register(MetricRegistry.name("workers","number"), + new Gauge[Int] { override def getValue: Int = master.workers.size - }) - - // Gauge for application numbers in cluster - gauges.put(className + ".apps.number", new Gauge[Int] { - override def getValue: Int = master.apps.size - }) - - // Gauge for waiting application numbers in cluster - gauges.put(className + ".waiting_apps.number", new Gauge[Int] { + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name("apps", "number"), + new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), + new Gauge[Int] { override def getValue: Int = master.waitingApps.size - }) - - gauges - } -} \ No newline at end of file + }) + +} diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 37fd154859..5ce29cf04c 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -4,86 +4,42 @@ import com.codahale.metrics.{Gauge, Metric} import java.util.{Map, HashMap => JHashMap} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import spark.metrics.source.Source private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { val className = classOf[Worker].getName() - override def sourceName = "worker" - - override def getMetrics: Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - // Gauge for executors number - gauges.put(className + ".executor.number", new Gauge[Int]{ + val sourceName = "worker" + + val metricRegistry = new MetricRegistry() + + metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + new Gauge[Int] { override def getValue: Int = worker.executors.size - }) - - gauges.put(className + ".core_used.number", new Gauge[Int]{ + }) + + // Gauge for cores used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + new Gauge[Int] { override def getValue: Int = worker.coresUsed - }) - - gauges.put(className + ".mem_used.MBytes", new Gauge[Int]{ + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), + new Gauge[Int] { override def getValue: Int = worker.memoryUsed - }) - - gauges.put(className + ".core_free.number", new Gauge[Int]{ + }) + + // Gauge for cores free of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + new Gauge[Int] { override def getValue: Int = worker.coresFree - }) - - gauges.put(className + ".mem_free.MBytes", new Gauge[Int]{ + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + new Gauge[Int] { override def getValue: Int = worker.memoryFree }) - - gauges - } } -//private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { -// var workerInst: Option[Worker] = None -// val metricRegistry = new MetricRegistry() -// -// override def registryHandler = metricRegistry -// -// override def instance = "worker" -// -// def initialize(worker: Worker) { -// workerInst = Some(worker) -// -// registerSources() -// registerSinks() -// } -// -// def uninitialize() { -// unregisterSinks() -// } -// -// // Gauge for executors number -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) -// }) -// -// // Gauge for cores used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) -// }) -// -// // Gauge for memory used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) -// }) -// -// // Gauge for cores free of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) -// }) -// -// // Gauge for memory used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) -// }) -//} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index ea1bc490b5..a23ccd2692 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -5,6 +5,7 @@ import scala.collection.mutable import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties +//import java.util._ import java.util.concurrent.TimeUnit import spark.Logging @@ -19,10 +20,13 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] + var registry = new MetricRegistry() + + registerSources() + registerSinks() def start() { - registerSources() - registerSinks() + sinks.foreach(_.start) } def stop() { @@ -31,20 +35,20 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + registry.register(source.sourceName,source.metricRegistry) } def registerSources() { val instConfig = metricsConfig.getInstance(instance) val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { val source = Class.forName(classPath).newInstance() sources += source.asInstanceOf[Source] - MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + registerSource(source.asInstanceOf[Source]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) } @@ -56,7 +60,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) // Register JMX sink as a default sink - sinks += new JmxSink(MetricsSystem.registry) + sinks += new JmxSink(registry) // Register other sinks according to conf sinkConfigs.foreach { kv => @@ -68,19 +72,16 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } try { val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) - .newInstance(kv._2, MetricsSystem.registry) + .newInstance(kv._2, registry) sinks += sink.asInstanceOf[Sink] } catch { case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.start) } } private[spark] object MetricsSystem { - val registry = new MetricRegistry() - val DEFAULT_SINKS = Map( "console" -> "spark.metrics.sink.ConsoleSink", "csv" -> "spark.metrics.sink.CsvSink") @@ -96,4 +97,4 @@ private[spark] object MetricsSystem { "day" -> TimeUnit.DAYS) def createMetricsSystem(instance: String) = new MetricsSystem(instance) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index b49a211fb3..9cd17556fa 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -18,14 +18,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } - var reporter: ConsoleReporter = _ - - override def start() { - reporter = ConsoleReporter.forRegistry(registry) + var reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() - + + + override def start() { reporter.start(pollPeriod, pollUnit) } @@ -40,4 +39,4 @@ object ConsoleSink { val CONSOLE_KEY_PERIOD = "period" val CONSOLE_KEY_UNIT = "unit" -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 3f572c8e05..62e51be0dc 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -24,15 +24,14 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si case None => CsvSink.CSV_DEFAULT_DIR } - var reporter: CsvReporter = _ - - override def start() { - reporter = CsvReporter.forRegistry(registry) + var reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build(new File(pollDir)) - + + + override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index e223dc26e9..98b55f7b7f 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -3,10 +3,9 @@ package spark.metrics.sink import com.codahale.metrics.{JmxReporter, MetricRegistry} class JmxSink(registry: MetricRegistry) extends Sink { - var reporter: JmxReporter = _ + var reporter: JmxReporter = JmxReporter.forRegistry(registry).build() override def start() { - reporter = JmxReporter.forRegistry(registry).build() reporter.start() } @@ -14,4 +13,4 @@ class JmxSink(registry: MetricRegistry) extends Sink { reporter.stop() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 7a7c1b6ffb..13270dae3c 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -2,22 +2,16 @@ package spark.metrics.source import java.util.{Map, HashMap => JHashMap} -import com.codahale.metrics.Metric +import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} class JvmSource extends Source { - override def sourceName = "jvm" - - override def getMetrics(): Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - import scala.collection.JavaConversions._ + val sourceName = "jvm" + val metricRegistry = new MetricRegistry() + val gcMetricSet = new GarbageCollectorMetricSet - gcMetricSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) - val memGaugeSet = new MemoryUsageGaugeSet - memGaugeSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) - - gauges - } -} \ No newline at end of file + + metricRegistry.registerAll(gcMetricSet) + metricRegistry.registerAll(memGaugeSet) +} diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index edd59de46a..17cbe2f85a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,7 +1,9 @@ package spark.metrics.source import com.codahale.metrics.MetricSet +import com.codahale.metrics.MetricRegistry -trait Source extends MetricSet { +trait Source { def sourceName: String -} \ No newline at end of file + def metricRegistry: MetricRegistry +} From 7fb574bf666661fdf8a786de779f85efe2f15f0c Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:14:30 +0800 Subject: [PATCH 266/419] Code clean and remarshal --- .../scala/spark/deploy/master/Master.scala | 9 +++++---- .../deploy/master/MasterInstrumentation.scala | 11 ++--------- .../deploy/worker/WorkerInstrumentation.scala | 18 ++++++------------ .../scala/spark/metrics/MetricsSystem.scala | 5 ++--- .../scala/spark/metrics/sink/ConsoleSink.scala | 1 - .../scala/spark/metrics/sink/CsvSink.scala | 1 - .../main/scala/spark/metrics/sink/Sink.scala | 1 - 7 files changed, 15 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index e44f5e3168..cc0b2d4295 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -78,6 +78,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) Master.metricsSystem.registerSource(masterInstrumentation) + Master.metricsSystem.start() } override def postStop() { @@ -321,22 +322,22 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } + + override def postStop() { + Master.metricsSystem.stop() + } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r - private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) - - metricsSystem.start() actorSystem.awaitTermination() - metricsSystem.stop() } /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 46c90b94d2..61a561c955 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -1,19 +1,12 @@ package spark.deploy.master -import java.util.{Map, HashMap => JHashMap} - -import com.codahale.metrics.{Gauge, Metric} - -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} +import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -import spark.Logging private[spark] class MasterInstrumentation(val master: Master) extends Source { - val className = classOf[Master].getName() - val instrumentationName = "master" val metricRegistry = new MetricRegistry() - val sourceName = instrumentationName + val sourceName = "master" metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 5ce29cf04c..94c20a98c1 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -1,44 +1,38 @@ package spark.deploy.worker -import com.codahale.metrics.{Gauge, Metric} +import com.codahale.metrics.{Gauge, MetricRegistry} -import java.util.{Map, HashMap => JHashMap} - -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import spark.metrics.source.Source private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { - val className = classOf[Worker].getName() - val sourceName = "worker" - val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { override def getValue: Int = worker.executors.size }) // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { override def getValue: Int = worker.coresUsed }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), + metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryUsed }) // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { override def getValue: Int = worker.coresFree }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryFree }) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index a23ccd2692..5bfdc00eaf 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -5,7 +5,6 @@ import scala.collection.mutable import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties -//import java.util._ import java.util.concurrent.TimeUnit import spark.Logging @@ -20,7 +19,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] - var registry = new MetricRegistry() + val registry = new MetricRegistry() registerSources() registerSinks() @@ -35,7 +34,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - registry.register(source.sourceName,source.metricRegistry) + registry.register(source.sourceName, source.metricRegistry) } def registerSources() { diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 9cd17556fa..e2e4197d1d 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -23,7 +23,6 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend .convertRatesTo(TimeUnit.SECONDS) .build() - override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 62e51be0dc..c2d645331c 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -30,7 +30,6 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si .convertRatesTo(TimeUnit.SECONDS) .build(new File(pollDir)) - override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 9fef894fde..3ffdcbdaba 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -2,6 +2,5 @@ package spark.metrics.sink trait Sink { def start: Unit - def stop: Unit } \ No newline at end of file From 576528f0f916cc0434972cbbc9321e4dcec45628 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:25:48 +0800 Subject: [PATCH 267/419] Add dependency of Codahale's metrics library --- project/SparkBuild.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f3f67b57c8..825319d3de 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -180,6 +180,8 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "com.codahale.metrics" % "metrics-core" % "3.0.0", + "com.codahale.metrics" % "metrics-jvm" % "3.0.0" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { From 871bc1687eaeb59df24b4778c5992a5f7f105cc8 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:46:24 +0800 Subject: [PATCH 268/419] Add Executor instrumentation --- .../executor/ExecutorInstrumentation.scala | 35 +++++++++++++++++++ .../spark/executor/MesosExecutorBackend.scala | 10 +++++- .../executor/StandaloneExecutorBackend.scala | 12 +++++++ 3 files changed, 56 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/spark/executor/ExecutorInstrumentation.scala diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala new file mode 100644 index 0000000000..80aadb66b0 --- /dev/null +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -0,0 +1,35 @@ +package spark.executor + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import spark.metrics.source.Source + +class ExecutorInstrumentation(val executor: Option[Executor]) extends Source{ + val metricRegistry = new MetricRegistry() + val sourceName = "executor" + + // Gauge for executor thread pool's actively executing task counts + metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getActiveCount()).getOrElse(0) + }) + + // Gauge for executor thread pool's approximate total number of tasks that have been completed + metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), + new Gauge[Long] { + override def getValue: Long = executor.map(_.threadPool.getCompletedTaskCount()).getOrElse(0) + }) + + // Gauge for executor thread pool's current number of threads + metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getPoolSize()).getOrElse(0) + }) + + // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getMaximumPoolSize()).getOrElse(0) + }) + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 4961c42fad..8b6ab0c391 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -24,6 +24,7 @@ import spark.TaskState.TaskState import com.google.protobuf.ByteString import spark.{Utils, Logging} import spark.TaskState +import spark.metrics.MetricsSystem private[spark] class MesosExecutorBackend extends MesosExecutor @@ -32,6 +33,9 @@ private[spark] class MesosExecutorBackend var executor: Executor = null var driver: ExecutorDriver = null + + val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) + MesosExecutorBackend.metricsSystem.start() override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() @@ -79,13 +83,17 @@ private[spark] class MesosExecutorBackend override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {} - override def shutdown(d: ExecutorDriver) {} + override def shutdown(d: ExecutorDriver) { + MesosExecutorBackend.metricsSystem.stop() + } } /** * Entry point for Mesos executor. */ private[spark] object MesosExecutorBackend { + private val metricsSystem = MetricsSystem.createMetricsSystem("executor") + def main(args: Array[String]) { MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index f4003da732..6ef74cd2ff 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -24,6 +24,7 @@ import spark.util.AkkaUtils import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} +import spark.metrics.MetricsSystem import spark.scheduler.cluster._ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask @@ -45,6 +46,8 @@ private[spark] class StandaloneExecutorBackend( var executor: Executor = null var driver: ActorRef = null + + val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) override def preStart() { logInfo("Connecting to driver: " + driverUrl) @@ -52,6 +55,9 @@ private[spark] class StandaloneExecutorBackend( driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(driver) // Doesn't work with remote actors, but useful for testing + + StandaloneExecutorBackend.metricsSystem.registerSource(executorInstrumentation) + StandaloneExecutorBackend.metricsSystem.start() } override def receive = { @@ -81,9 +87,15 @@ private[spark] class StandaloneExecutorBackend( override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { driver ! StatusUpdate(executorId, taskId, state, data) } + + override def postStop() { + StandaloneExecutorBackend.metricsSystem.stop() + } } private[spark] object StandaloneExecutorBackend { + private val metricsSystem = MetricsSystem.createMetricsSystem("executor") + def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) } From 5ce5dc9fcd7acf5c58dd3d456a629b01d57514e4 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:48:21 +0800 Subject: [PATCH 269/419] Add default properties to deal with no configure file situation --- .../scala/spark/metrics/MetricsConfig.scala | 28 ++++++++++++------- .../scala/spark/metrics/MetricsSystem.scala | 9 ++---- .../spark/metrics/sink/ConsoleSink.scala | 6 ++-- .../scala/spark/metrics/sink/CsvSink.scala | 6 ++-- .../scala/spark/metrics/sink/JmxSink.scala | 6 ++-- 5 files changed, 30 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index be4f670918..7405192058 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -1,20 +1,25 @@ package spark.metrics import java.util.Properties -import java.io.FileInputStream +import java.io.{File, FileInputStream} import scala.collection.mutable import scala.util.matching.Regex private [spark] class MetricsConfig(val configFile: String) { val properties = new Properties() - var fis: FileInputStream = _ + // Add default properties in case there's no properties file + MetricsConfig.setDefaultProperties(properties) - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() + val confFile = new File(configFile) + if (confFile.exists()) { + var fis: FileInputStream = null + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } } val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) @@ -35,11 +40,15 @@ private [spark] class MetricsConfig(val configFile: String) { } } -object MetricsConfig { - val DEFAULT_CONFIG_FILE = "conf/metrics.properties" +private[spark] object MetricsConfig { val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + def setDefaultProperties(prop: Properties) { + prop.setProperty("*.sink.jmx.enabled", "default") + prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + } + def subProperties(prop: Properties, regex: Regex) = { val subProperties = new mutable.HashMap[String, Properties] @@ -48,7 +57,6 @@ object MetricsConfig { if (regex.findPrefixOf(kv._1) != None) { val regex(prefix, suffix) = kv._1 subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) - println(">>>>>subProperties added " + prefix + " " + suffix + " " + kv._2) } } diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 5bfdc00eaf..6e448cb2a5 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -14,7 +14,7 @@ import spark.metrics.source._ private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) + val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] @@ -58,9 +58,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - // Register JMX sink as a default sink - sinks += new JmxSink(registry) - // Register other sinks according to conf sinkConfigs.foreach { kv => val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { @@ -81,9 +78,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } private[spark] object MetricsSystem { - val DEFAULT_SINKS = Map( - "console" -> "spark.metrics.sink.ConsoleSink", - "csv" -> "spark.metrics.sink.CsvSink") + val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index e2e4197d1d..d7b7a9e501 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -1,10 +1,10 @@ package spark.metrics.sink +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + import java.util.Properties import java.util.concurrent.TimeUnit -import com.codahale.metrics.{ConsoleReporter, MetricRegistry} - import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { @@ -18,7 +18,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } - var reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index c2d645331c..e6c5bffd3c 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -1,11 +1,11 @@ package spark.metrics.sink +import com.codahale.metrics.{CsvReporter, MetricRegistry} + import java.io.File import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit -import com.codahale.metrics.{CsvReporter, MetricRegistry} - import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { @@ -24,7 +24,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si case None => CsvSink.CSV_DEFAULT_DIR } - var reporter: CsvReporter = CsvReporter.forRegistry(registry) + val reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index 98b55f7b7f..f097a631c0 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -1,9 +1,11 @@ package spark.metrics.sink +import java.util.Properties + import com.codahale.metrics.{JmxReporter, MetricRegistry} -class JmxSink(registry: MetricRegistry) extends Sink { - var reporter: JmxReporter = JmxReporter.forRegistry(registry).build() +class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() override def start() { reporter.start() From e080588f7396d9612ea5d909e59f2364c139103a Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:04:06 +0800 Subject: [PATCH 270/419] Add metrics system unit test --- .../spark/metrics/MetricsConfigSuite.scala | 92 +++++++++++++++++++ .../spark/metrics/MetricsSystemSuite.scala | 65 +++++++++++++ 2 files changed, 157 insertions(+) create mode 100644 core/src/test/scala/spark/metrics/MetricsConfigSuite.scala create mode 100644 core/src/test/scala/spark/metrics/MetricsSystemSuite.scala diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala new file mode 100644 index 0000000000..0c7142c418 --- /dev/null +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -0,0 +1,92 @@ +package spark.metrics + +import java.util.Properties +import java.io.{File, FileOutputStream} + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import spark.metrics._ + +class MetricsConfigSuite extends FunSuite with BeforeAndAfter { + var filePath: String = _ + + before { + val prop = new Properties() + + prop.setProperty("*.sink.console.period", "10") + prop.setProperty("*.sink.console.unit", "second") + prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + prop.setProperty("master.sink.console.period", "20") + prop.setProperty("master.sink.console.unit", "minute") + + val dir = new File("/tmp") + filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { + "/tmp/test_metrics.properties" + } else { + "./test_metrics.properties" + } + + val os = new FileOutputStream(new File(filePath)) + prop.store(os, "for test") + os.close() + } + + test("MetricsConfig with default properties") { + val conf = new MetricsConfig("dummy-file") + assert(conf.properties.size() === 2) + assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") + assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") + assert(conf.properties.getProperty("test-for-dummy") === null) + + val property = conf.getInstance("random") + assert(property.size() === 2) + assert(property.getProperty("sink.jmx.enabled") === "default") + assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") + } + + test("MetricsConfig with properties set") { + val conf = new MetricsConfig(filePath) + + val masterProp = conf.getInstance("master") + assert(masterProp.size() === 4) + assert(masterProp.getProperty("sink.console.period") === "20") + assert(masterProp.getProperty("sink.console.unit") === "minute") + assert(masterProp.getProperty("sink.jmx.enabled") === "default") + assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") + + val workerProp = conf.getInstance("worker") + assert(workerProp.size() === 4) + assert(workerProp.getProperty("sink.console.period") === "10") + assert(workerProp.getProperty("sink.console.unit") === "second") + } + + test("MetricsConfig with subProperties") { + val conf = new MetricsConfig(filePath) + + val propCategories = conf.propertyCategories + assert(propCategories.size === 2) + + val masterProp = conf.getInstance("master") + val sourceProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) + assert(sourceProps.size === 1) + assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") + + val sinkProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SINK_REGEX) + assert(sinkProps.size === 2) + assert(sinkProps.contains("console")) + assert(sinkProps.contains("jmx")) + + val consoleProps = sinkProps("console") + assert(consoleProps.size() === 2) + + val jmxProps = sinkProps("jmx") + assert(jmxProps.size() === 1) + } + + after { + val file = new File(filePath) + if (file.exists()) { + file.delete() + } + } +} \ No newline at end of file diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala new file mode 100644 index 0000000000..5e8f8fcf80 --- /dev/null +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -0,0 +1,65 @@ +package spark.metrics + +import java.util.Properties +import java.io.{File, FileOutputStream} + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import spark.metrics._ + +class MetricsSystemSuite extends FunSuite with BeforeAndAfter { + var filePath: String = _ + + before { + val props = new Properties() + props.setProperty("*.sink.console.period", "10") + props.setProperty("*.sink.console.unit", "second") + props.setProperty("test.sink.console.class", "spark.metrics.sink.ConsoleSink") + props.setProperty("test.sink.dummy.class", "spark.metrics.sink.DummySink") + props.setProperty("test.source.dummy.class", "spark.metrics.source.DummySource") + props.setProperty("test.sink.console.period", "20") + props.setProperty("test.sink.console.unit", "minute") + + val dir = new File("/tmp") + filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { + "/tmp/test_metrics.properties" + } else { + "./test_metrics.properties" + } + + val os = new FileOutputStream(new File(filePath)) + props.store(os, "for test") + os.close() + System.setProperty("spark.metrics.conf.file", filePath) + } + + test("MetricsSystem with default config") { + val metricsSystem = MetricsSystem.createMetricsSystem("default") + val sources = metricsSystem.sources + val sinks = metricsSystem.sinks + + assert(sources.length === 1) + assert(sinks.length === 1) + assert(sources(0).sourceName === "jvm") + } + + test("MetricsSystem with sources add") { + val metricsSystem = MetricsSystem.createMetricsSystem("test") + val sources = metricsSystem.sources + val sinks = metricsSystem.sinks + + assert(sources.length === 1) + assert(sinks.length === 2) + + val source = new spark.deploy.master.MasterInstrumentation(null) + metricsSystem.registerSource(source) + assert(sources.length === 2) + } + + after { + val file = new File(filePath) + if (file.exists()) { + file.delete() + } + } +} \ No newline at end of file From e9ac88754d4c5d58aedd4de8768787300b15eada Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:05:18 +0800 Subject: [PATCH 271/419] Remove twice add Source bug and code clean --- .../src/main/scala/spark/metrics/MetricsSystem.scala | 10 ++++------ .../main/scala/spark/metrics/source/JvmSource.scala | 12 ++++++------ 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 6e448cb2a5..bf4487e0fc 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -1,15 +1,15 @@ package spark.metrics -import scala.collection.mutable - import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit +import scala.collection.mutable + import spark.Logging -import spark.metrics.sink._ -import spark.metrics.source._ +import spark.metrics.sink.Sink +import spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() @@ -46,7 +46,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val classPath = kv._2.getProperty("class") try { val source = Class.forName(classPath).newInstance() - sources += source.asInstanceOf[Source] registerSource(source.asInstanceOf[Source]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) @@ -58,7 +57,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - // Register other sinks according to conf sinkConfigs.foreach { kv => val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { MetricsSystem.DEFAULT_SINKS(kv._1) diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 13270dae3c..8f0870c1a0 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -1,17 +1,17 @@ package spark.metrics.source -import java.util.{Map, HashMap => JHashMap} - import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} +import java.util.{Map, HashMap => JHashMap} + class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() - val gcMetricSet = new GarbageCollectorMetricSet - val memGaugeSet = new MemoryUsageGaugeSet + val gcMetricSet = new GarbageCollectorMetricSet + val memGaugeSet = new MemoryUsageGaugeSet - metricRegistry.registerAll(gcMetricSet) - metricRegistry.registerAll(memGaugeSet) + metricRegistry.registerAll(gcMetricSet) + metricRegistry.registerAll(memGaugeSet) } From 7d2eada451686824bd467641bf1763e82011f2a6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 21:37:21 +0800 Subject: [PATCH 272/419] Add metrics source of DAGScheduler and blockManager Conflicts: core/src/main/scala/spark/SparkContext.scala core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkContext.scala | 13 +++++-- core/src/main/scala/spark/SparkEnv.scala | 6 ++++ .../scala/spark/scheduler/DAGScheduler.scala | 3 +- .../spark/scheduler/DAGSchedulerSource.scala | 31 ++++++++++++++++ .../spark/storage/BlockManagerSource.scala | 35 +++++++++++++++++++ 5 files changed, 84 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala create mode 100644 core/src/main/scala/spark/storage/BlockManagerSource.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 24ba605646..a6128a9f30 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -60,11 +60,11 @@ import org.apache.mesos.MesosNativeLibrary import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} +import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import spark.storage.{StorageStatus, StorageUtils, RDDInfo} +import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} @@ -270,6 +270,15 @@ class SparkContext( // Post init taskScheduler.postStartHook() + val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) + val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) + def initDriverMetrics() = { + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + } + + initDriverMetrics() + // Methods for creating RDDs /** Distribute a local Scala collection to form an RDD. */ diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index f2bdc11bdb..d34dafecc5 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -30,6 +30,7 @@ import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils import spark.api.python.PythonWorkerFactory +import spark.metrics._ /** @@ -53,6 +54,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, + val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -184,6 +186,9 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + val metricsSystem = MetricsSystem.createMetricsSystem("driver") + metricsSystem.start() + // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working // directory. @@ -213,6 +218,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, + metricsSystem, None) } } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 9b45fc2938..781e49bdec 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -30,7 +30,7 @@ import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialRe import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} - +import spark.metrics.MetricsSystem /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal @@ -126,7 +126,6 @@ class DAGScheduler( val resultStageToJob = new HashMap[Stage, ActiveJob] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) - // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala new file mode 100644 index 0000000000..57aa74512c --- /dev/null +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -0,0 +1,31 @@ +package spark.scheduler + +import com.codahale.metrics.{Gauge,MetricRegistry} + +import spark.metrics.source.Source + +private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "DAGScheduler" + + + metricRegistry.register(MetricRegistry.name("stage","failedStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.failed.size + }) + + metricRegistry.register(MetricRegistry.name("stage","runningStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.running.size + }) + + metricRegistry.register(MetricRegistry.name("stage","waitingStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.waiting.size + }) + + metricRegistry.register(MetricRegistry.name("job","allJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.nextRunId.get() + }) + + metricRegistry.register(MetricRegistry.name("job","ActiveJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.activeJobs.size + }) +} diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala new file mode 100644 index 0000000000..c0ce9259c8 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -0,0 +1,35 @@ +package spark.storage + +import com.codahale.metrics.{Gauge,MetricRegistry} + +import spark.metrics.source.Source +import spark.storage._ + +private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "BlockManager" + + metricRegistry.register(MetricRegistry.name("memory","maxMem"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + maxMem + } + }) + + metricRegistry.register(MetricRegistry.name("memory","remainingMem"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + remainingMem + } + }) + + metricRegistry.register(MetricRegistry.name("disk","diskSpaceUsed"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) + diskSpaceUsed + } + }) +} From 9cea0c28184c86625f8281eea2af77eff15acb73 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 22:22:29 +0800 Subject: [PATCH 273/419] Refactor metricsSystem unit test, add resource files. --- .../resources/test_metrics_config.properties | 6 ++++ .../resources/test_metrics_system.properties | 7 +++++ .../spark/metrics/MetricsConfigSuite.scala | 30 ++---------------- .../spark/metrics/MetricsSystemSuite.scala | 31 ++----------------- 4 files changed, 19 insertions(+), 55 deletions(-) create mode 100644 core/src/test/resources/test_metrics_config.properties create mode 100644 core/src/test/resources/test_metrics_system.properties diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties new file mode 100644 index 0000000000..2011940003 --- /dev/null +++ b/core/src/test/resources/test_metrics_config.properties @@ -0,0 +1,6 @@ +*.sink.console.period = 10 +*.sink.console.unit = second +*.source.jvm.class = spark.metrics.source.JvmSource +master.sink.console.period = 20 +master.sink.console.unit = minute + diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties new file mode 100644 index 0000000000..06afbc6625 --- /dev/null +++ b/core/src/test/resources/test_metrics_system.properties @@ -0,0 +1,7 @@ +*.sink.console.period = 10 +*.sink.console.unit = second +test.sink.console.class = spark.metrics.sink.ConsoleSink +test.sink.dummy.class = spark.metrics.sink.DummySink +test.source.dummy.class = spark.metrics.source.DummySource +test.sink.console.period = 20 +test.sink.console.unit = minute diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index 0c7142c418..f4c83cb644 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -11,24 +11,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ before { - val prop = new Properties() - - prop.setProperty("*.sink.console.period", "10") - prop.setProperty("*.sink.console.unit", "second") - prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") - prop.setProperty("master.sink.console.period", "20") - prop.setProperty("master.sink.console.unit", "minute") - - val dir = new File("/tmp") - filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { - "/tmp/test_metrics.properties" - } else { - "./test_metrics.properties" - } - - val os = new FileOutputStream(new File(filePath)) - prop.store(os, "for test") - os.close() + filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile() } test("MetricsConfig with default properties") { @@ -81,12 +64,5 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { val jmxProps = sinkProps("jmx") assert(jmxProps.size() === 1) - } - - after { - val file = new File(filePath) - if (file.exists()) { - file.delete() - } - } -} \ No newline at end of file + } +} diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 5e8f8fcf80..967be6ec47 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -11,25 +11,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ before { - val props = new Properties() - props.setProperty("*.sink.console.period", "10") - props.setProperty("*.sink.console.unit", "second") - props.setProperty("test.sink.console.class", "spark.metrics.sink.ConsoleSink") - props.setProperty("test.sink.dummy.class", "spark.metrics.sink.DummySink") - props.setProperty("test.source.dummy.class", "spark.metrics.source.DummySource") - props.setProperty("test.sink.console.period", "20") - props.setProperty("test.sink.console.unit", "minute") - - val dir = new File("/tmp") - filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { - "/tmp/test_metrics.properties" - } else { - "./test_metrics.properties" - } - - val os = new FileOutputStream(new File(filePath)) - props.store(os, "for test") - os.close() + filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf.file", filePath) } @@ -54,12 +36,5 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val source = new spark.deploy.master.MasterInstrumentation(null) metricsSystem.registerSource(source) assert(sources.length === 2) - } - - after { - val file = new File(filePath) - if (file.exists()) { - file.delete() - } - } -} \ No newline at end of file + } +} From 5f8802c1fb106cc04c30b6aca0a6ce98fa5c0e15 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 2 Jul 2013 10:41:43 +0800 Subject: [PATCH 274/419] Register and init metricsSystem in SparkContext Conflicts: core/src/main/scala/spark/SparkContext.scala core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkContext.scala | 10 ++++++++-- core/src/main/scala/spark/SparkEnv.scala | 4 ---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index a6128a9f30..1255d0c72e 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -67,6 +67,9 @@ import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} +import spark.metrics._ + +import scala.util.DynamicVariable /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -272,9 +275,12 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) + val metricsSystem = MetricsSystem.createMetricsSystem("driver") + def initDriverMetrics() = { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + metricsSystem.registerSource(dagSchedulerSource) + metricsSystem.registerSource(blockManagerSource) + metricsSystem.start() } initDriverMetrics() diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index d34dafecc5..125dc55bd8 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -54,7 +54,6 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -186,8 +185,6 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) - val metricsSystem = MetricsSystem.createMetricsSystem("driver") - metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working @@ -218,7 +215,6 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - metricsSystem, None) } } From 1daff54b2ed92d0bcee7030d7d3ab5c274f80d2f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:28:32 +0800 Subject: [PATCH 275/419] Change Executor MetricsSystem initialize code to SparkEnv --- core/src/main/scala/spark/SparkContext.scala | 6 ++---- core/src/main/scala/spark/SparkEnv.scala | 9 +++++++++ core/src/main/scala/spark/executor/Executor.scala | 6 +++++- .../spark/executor/ExecutorInstrumentation.scala | 10 +++++----- .../scala/spark/executor/MesosExecutorBackend.scala | 10 +--------- .../spark/executor/StandaloneExecutorBackend.scala | 12 ------------ 6 files changed, 22 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1255d0c72e..f1d9d5e442 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -275,12 +275,10 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - val metricsSystem = MetricsSystem.createMetricsSystem("driver") def initDriverMetrics() = { - metricsSystem.registerSource(dagSchedulerSource) - metricsSystem.registerSource(blockManagerSource) - metricsSystem.start() + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) } initDriverMetrics() diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 125dc55bd8..7b3dc69348 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -54,6 +54,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, + val metricsSystem: metricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -69,6 +70,7 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() + metricsSystem.stop() actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release @@ -185,6 +187,12 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + val metricsSystem = if (isDriver) { + MetricsSystem.createMetricsSystem("driver") + } else { + MetricsSystem.createMetricsSystem("executor") + } + metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working @@ -215,6 +223,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, + metricsSystem, None) } } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 2e81151882..7179ed84a8 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -86,10 +86,14 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } } ) + + val executorInstrumentation = new ExecutorInstrumentation(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) - SparkEnv.set(env) + SparkEnv.set(env) + env.metricsSystem.registerSource(executorInstrumentation) + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala index 80aadb66b0..ebbcbee742 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -4,32 +4,32 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Option[Executor]) extends Source{ +class ExecutorInstrumentation(val executor: Executor) extends Source{ val metricRegistry = new MetricRegistry() val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getActiveCount()).getOrElse(0) + override def getValue: Int = executor.threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { - override def getValue: Long = executor.map(_.threadPool.getCompletedTaskCount()).getOrElse(0) + override def getValue: Long = executor.threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getPoolSize()).getOrElse(0) + override def getValue: Int = executor.threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getMaximumPoolSize()).getOrElse(0) + override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) } \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 8b6ab0c391..4961c42fad 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -24,7 +24,6 @@ import spark.TaskState.TaskState import com.google.protobuf.ByteString import spark.{Utils, Logging} import spark.TaskState -import spark.metrics.MetricsSystem private[spark] class MesosExecutorBackend extends MesosExecutor @@ -33,9 +32,6 @@ private[spark] class MesosExecutorBackend var executor: Executor = null var driver: ExecutorDriver = null - - val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) - MesosExecutorBackend.metricsSystem.start() override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() @@ -83,17 +79,13 @@ private[spark] class MesosExecutorBackend override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {} - override def shutdown(d: ExecutorDriver) { - MesosExecutorBackend.metricsSystem.stop() - } + override def shutdown(d: ExecutorDriver) {} } /** * Entry point for Mesos executor. */ private[spark] object MesosExecutorBackend { - private val metricsSystem = MetricsSystem.createMetricsSystem("executor") - def main(args: Array[String]) { MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 6ef74cd2ff..f4003da732 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -24,7 +24,6 @@ import spark.util.AkkaUtils import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} -import spark.metrics.MetricsSystem import spark.scheduler.cluster._ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask @@ -46,8 +45,6 @@ private[spark] class StandaloneExecutorBackend( var executor: Executor = null var driver: ActorRef = null - - val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) override def preStart() { logInfo("Connecting to driver: " + driverUrl) @@ -55,9 +52,6 @@ private[spark] class StandaloneExecutorBackend( driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(driver) // Doesn't work with remote actors, but useful for testing - - StandaloneExecutorBackend.metricsSystem.registerSource(executorInstrumentation) - StandaloneExecutorBackend.metricsSystem.start() } override def receive = { @@ -87,15 +81,9 @@ private[spark] class StandaloneExecutorBackend( override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { driver ! StatusUpdate(executorId, taskId, state, data) } - - override def postStop() { - StandaloneExecutorBackend.metricsSystem.stop() - } } private[spark] object StandaloneExecutorBackend { - private val metricsSystem = MetricsSystem.createMetricsSystem("executor") - def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) } From a79f6077f0b3bc0110f6e8e15de3068f296f3b81 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:29:26 +0800 Subject: [PATCH 276/419] Add Maven metrics library dependency and code changes --- conf/metrics.properties | 4 ++-- core/pom.xml | 8 ++++++++ core/src/main/scala/spark/SparkEnv.scala | 3 ++- pom.xml | 8 ++++++++ 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/conf/metrics.properties b/conf/metrics.properties index 0bbb6b5229..2ea3aaf345 100644 --- a/conf/metrics.properties +++ b/conf/metrics.properties @@ -1,10 +1,10 @@ -# syntax: [prefix].[sink|source].[instance].[options] +# syntax: [instance].[sink|source].[name].[options] *.sink.console.period=10 *.sink.console.unit=second -master.source.jvm.class=spark.metrics.source.JvmSource +*.sink.console.class=spark.metrics.sink.ConsoleSink master.sink.console.period=10 diff --git a/core/pom.xml b/core/pom.xml index 6329b2fbd8..237d988712 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -108,6 +108,14 @@ log4j log4j + + com.codahale.metrics + metrics-core + + + com.codahale.metrics + metrics-jvm + org.apache.derby diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7b3dc69348..204049d927 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -24,6 +24,7 @@ import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider import spark.broadcast.BroadcastManager +import spark.metrics.MetricsSystem import spark.storage.BlockManager import spark.storage.BlockManagerMaster import spark.network.ConnectionManager @@ -54,7 +55,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: metricsSystem, + val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort diff --git a/pom.xml b/pom.xml index eb7bd7e9df..3fbd93c7dd 100644 --- a/pom.xml +++ b/pom.xml @@ -268,6 +268,14 @@ org.scala-lang scalap ${scala.version} + com.codahale.metrics + metrics-core + 3.0.0 + + + com.codahale.metrics + metrics-jvm + 3.0.0 From 5730193e0c8639b654f489c5956e31451d81b7db Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 13:48:09 +0800 Subject: [PATCH 277/419] Fix some typos --- core/src/main/scala/spark/SparkContext.scala | 2 -- core/src/main/scala/spark/SparkEnv.scala | 1 - core/src/main/scala/spark/metrics/sink/JmxSink.scala | 10 +++++----- .../main/scala/spark/metrics/source/JvmSource.scala | 6 ++---- core/src/main/scala/spark/metrics/source/Source.scala | 3 +-- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 3 ++- project/SparkBuild.scala | 2 +- 7 files changed, 11 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index f1d9d5e442..1e59a4d47d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -69,8 +69,6 @@ import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} import spark.metrics._ -import scala.util.DynamicVariable - /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 204049d927..4a1d341f5d 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -31,7 +31,6 @@ import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils import spark.api.python.PythonWorkerFactory -import spark.metrics._ /** diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index f097a631c0..6a40885b78 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -1,18 +1,18 @@ package spark.metrics.sink -import java.util.Properties - import com.codahale.metrics.{JmxReporter, MetricRegistry} +import java.util.Properties + class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() - + override def start() { reporter.start() } - + override def stop() { reporter.stop() } - + } diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 8f0870c1a0..79f505079c 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -3,15 +3,13 @@ package spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} -import java.util.{Map, HashMap => JHashMap} - class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() - + val gcMetricSet = new GarbageCollectorMetricSet val memGaugeSet = new MemoryUsageGaugeSet - + metricRegistry.registerAll(gcMetricSet) metricRegistry.registerAll(memGaugeSet) } diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index 17cbe2f85a..5607e2c40a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,9 +1,8 @@ package spark.metrics.source -import com.codahale.metrics.MetricSet import com.codahale.metrics.MetricRegistry trait Source { def sourceName: String - def metricRegistry: MetricRegistry + def metricRegistry: MetricRegistry } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 781e49bdec..9b45fc2938 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -30,7 +30,7 @@ import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialRe import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import spark.metrics.MetricsSystem + /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal @@ -126,6 +126,7 @@ class DAGScheduler( val resultStageToJob = new HashMap[Stage, ActiveJob] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) + // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 825319d3de..d4d70afdd5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -179,7 +179,7 @@ object SparkBuild extends Build { "net.liftweb" % "lift-json_2.9.2" % "2.5", "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", - "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0" ) ++ ( From ed1a3bc206c01974eedd0b1fb1deec183376b5c6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:11:10 +0800 Subject: [PATCH 278/419] continue to refactor code style and functions --- core/src/main/scala/spark/SparkContext.scala | 6 +- .../scala/spark/deploy/master/Master.scala | 6 +- .../deploy/master/MasterInstrumentation.scala | 19 ++-- .../deploy/worker/WorkerInstrumentation.scala | 35 ++++---- .../main/scala/spark/executor/Executor.scala | 8 +- .../executor/ExecutorInstrumentation.scala | 47 +++++----- .../scala/spark/metrics/MetricsConfig.scala | 88 +++++++++---------- .../scala/spark/metrics/MetricsSystem.scala | 40 ++++----- .../spark/metrics/sink/ConsoleSink.scala | 30 +++---- .../scala/spark/metrics/sink/CsvSink.scala | 40 ++++----- .../spark/scheduler/DAGSchedulerSource.scala | 23 +++-- .../spark/storage/BlockManagerSource.scala | 14 +-- .../spark/metrics/MetricsConfigSuite.scala | 32 ++++--- .../spark/metrics/MetricsSystemSuite.scala | 14 +-- 14 files changed, 195 insertions(+), 207 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1e59a4d47d..77cb0ee0cd 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -274,9 +274,9 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - def initDriverMetrics() = { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + def initDriverMetrics() { + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) } initDriverMetrics() diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index cc0b2d4295..5f67366eb6 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -57,7 +57,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") - + val masterInstrumentation = new MasterInstrumentation(this) val masterPublicAddress = { @@ -76,7 +76,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - + Master.metricsSystem.registerSource(masterInstrumentation) Master.metricsSystem.start() } @@ -322,7 +322,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } - + override def postStop() { Master.metricsSystem.stop() } diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 61a561c955..4c3708cc4c 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -5,24 +5,21 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source private[spark] class MasterInstrumentation(val master: Master) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "master" - metricRegistry.register(MetricRegistry.name("workers","number"), - new Gauge[Int] { + // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { override def getValue: Int = master.workers.size }) - + // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name("apps", "number"), - new Gauge[Int] { - override def getValue: Int = master.apps.size + metricRegistry.register(MetricRegistry.name("apps", "number"), new Gauge[Int] { + override def getValue: Int = master.apps.size }) // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), - new Gauge[Int] { - override def getValue: Int = master.waitingApps.size + metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { + override def getValue: Int = master.waitingApps.size }) - } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 94c20a98c1..c76c0b4711 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -8,32 +8,27 @@ private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name("executor", "number"), - new Gauge[Int] { - override def getValue: Int = worker.executors.size + metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { + override def getValue: Int = worker.executors.size }) - + // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name("core_used", "number"), - new Gauge[Int] { - override def getValue: Int = worker.coresUsed + metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresUsed }) - + // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), - new Gauge[Int] { - override def getValue: Int = worker.memoryUsed + metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryUsed }) - + // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name("core_free", "number"), - new Gauge[Int] { - override def getValue: Int = worker.coresFree + metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresFree }) - + // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), - new Gauge[Int] { - override def getValue: Int = worker.memoryFree - }) + metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryFree + }) } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 7179ed84a8..4ea05dec1c 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -69,7 +69,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert override def uncaughtException(thread: Thread, exception: Throwable) { try { logError("Uncaught exception in thread " + thread, exception) - + // We may have been called from a shutdown hook. If so, we must not call System.exit(). // (If we do, we will deadlock.) if (!Utils.inShutdown()) { @@ -86,14 +86,14 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } } ) - + val executorInstrumentation = new ExecutorInstrumentation(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) - SparkEnv.set(env) + SparkEnv.set(env) env.metricsSystem.registerSource(executorInstrumentation) - + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala index ebbcbee742..ad406f41b4 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -4,32 +4,27 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Executor) extends Source{ +class ExecutorInstrumentation(val executor: Executor) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "executor" - - // Gauge for executor thread pool's actively executing task counts - metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getActiveCount() - }) - - // Gauge for executor thread pool's approximate total number of tasks that have been completed - metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), - new Gauge[Long] { - override def getValue: Long = executor.threadPool.getCompletedTaskCount() - }) - - // Gauge for executor thread pool's current number of threads - metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getPoolSize() - }) - - // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool - metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getMaximumPoolSize() - }) -} \ No newline at end of file + // Gauge for executor thread pool's actively executing task counts + metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getActiveCount() + }) + + // Gauge for executor thread pool's approximate total number of tasks that have been completed + metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { + override def getValue: Long = executor.threadPool.getCompletedTaskCount() + }) + + // Gauge for executor thread pool's current number of threads + metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getPoolSize() + }) + + // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getMaximumPoolSize() + }) +} diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 7405192058..b1f6a1e596 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -3,63 +3,63 @@ package spark.metrics import java.util.Properties import java.io.{File, FileInputStream} -import scala.collection.mutable +import scala.collection.mutable.HashMap import scala.util.matching.Regex -private [spark] class MetricsConfig(val configFile: String) { +private[spark] class MetricsConfig(val configFile: String) { val properties = new Properties() - // Add default properties in case there's no properties file - MetricsConfig.setDefaultProperties(properties) - - val confFile = new File(configFile) - if (confFile.exists()) { - var fis: FileInputStream = null - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() - } - } - - val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) - if (propertyCategories.contains(MetricsConfig.DEFAULT_PREFIX)) { - import scala.collection.JavaConversions._ - val defaultProperty = propertyCategories(MetricsConfig.DEFAULT_PREFIX) - for ((inst, prop) <- propertyCategories; p <- defaultProperty - if inst != MetricsConfig.DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { - prop.setProperty(p._1, p._2) - } - } - - def getInstance(inst: String) = { - propertyCategories.get(inst) match { - case Some(s) => s - case None => propertyCategories(MetricsConfig.DEFAULT_PREFIX) - } - } -} - -private[spark] object MetricsConfig { val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r - - def setDefaultProperties(prop: Properties) { + var propertyCategories: HashMap[String, Properties] = null + + private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.jmx.enabled", "default") prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") } - - def subProperties(prop: Properties, regex: Regex) = { - val subProperties = new mutable.HashMap[String, Properties] - + + def initilize() { + //Add default properties in case there's no properties file + setDefaultProperties(properties) + + val confFile = new File(configFile) + if (confFile.exists()) { + var fis: FileInputStream = null + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } + } + + propertyCategories = subProperties(properties, INSTANCE_REGEX) + if (propertyCategories.contains(DEFAULT_PREFIX)) { + import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(DEFAULT_PREFIX) + for ((inst, prop) <- propertyCategories; p <- defaultProperty + if inst != DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { + prop.setProperty(p._1, p._2) + } + } + } + + def subProperties(prop: Properties, regex: Regex): HashMap[String, Properties] = { + val subProperties = new HashMap[String, Properties] import scala.collection.JavaConversions._ - prop.foreach { kv => + prop.foreach { kv => if (regex.findPrefixOf(kv._1) != None) { val regex(prefix, suffix) = kv._1 subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) } } - subProperties } -} \ No newline at end of file + + def getInstance(inst: String): Properties = { + propertyCategories.get(inst) match { + case Some(s) => s + case None => propertyCategories(DEFAULT_PREFIX) + } + } +} + diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index bf4487e0fc..54f6c6e4da 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -13,34 +13,37 @@ import spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - + val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") val metricsConfig = new MetricsConfig(confFile) - + val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] val registry = new MetricRegistry() - + + val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") + + metricsConfig.initilize() registerSources() registerSinks() - + def start() { sinks.foreach(_.start) } - + def stop() { sinks.foreach(_.stop) } - + def registerSource(source: Source) { sources += source registry.register(source.sourceName, source.metricRegistry) } - + def registerSources() { val instConfig = metricsConfig.getInstance(instance) - val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - + val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") @@ -52,14 +55,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } } } - + def registerSinks() { val instConfig = metricsConfig.getInstance(instance) - val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - + val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) + sinkConfigs.foreach { kv => - val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { - MetricsSystem.DEFAULT_SINKS(kv._1) + val classPath = if (DEFAULT_SINKS.contains(kv._1)) { + DEFAULT_SINKS(kv._1) } else { // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") @@ -76,17 +79,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } private[spark] object MetricsSystem { - val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") - val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val timeUnits = Map( - "millisecond" -> TimeUnit.MILLISECONDS, + "illisecond" -> TimeUnit.MILLISECONDS, "second" -> TimeUnit.SECONDS, "minute" -> TimeUnit.MINUTES, "hour" -> TimeUnit.HOURS, "day" -> TimeUnit.DAYS) - - def createMetricsSystem(instance: String) = new MetricsSystem(instance) + + def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) } diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index d7b7a9e501..c67c0ee912 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,34 +8,34 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { + + val CONSOLE_DEFAULT_PERIOD = "10" + val CONSOLE_DEFAULT_UNIT = "second" + + val CONSOLE_KEY_PERIOD = "period" + val CONSOLE_KEY_UNIT = "unit" + + val pollPeriod = Option(property.getProperty(CONSOLE_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => ConsoleSink.CONSOLE_DEFAULT_PERIOD.toInt + case None => CONSOLE_DEFAULT_PERIOD.toInt } - - val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { + + val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + case None => MetricsSystem.timeUnits(CONSOLE_DEFAULT_UNIT) } - + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() override def start() { - reporter.start(pollPeriod, pollUnit) + reporter.start(pollPeriod, pollUnit) } - + override def stop() { reporter.stop() } } -object ConsoleSink { - val CONSOLE_DEFAULT_PERIOD = "10" - val CONSOLE_DEFAULT_UNIT = "second" - - val CONSOLE_KEY_PERIOD = "period" - val CONSOLE_KEY_UNIT = "unit" -} diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index e6c5bffd3c..a8ca819e87 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -9,21 +9,29 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { + val CSV_KEY_PERIOD = "period" + val CSV_KEY_UNIT = "unit" + val CSV_KEY_DIR = "directory" + + val CSV_DEFAULT_PERIOD = "10" + val CSV_DEFAULT_UNIT = "second" + val CSV_DEFAULT_DIR = "/tmp/" + + val pollPeriod = Option(property.getProperty(CSV_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CsvSink.CSV_DEFAULT_PERIOD.toInt + case None => CSV_DEFAULT_PERIOD.toInt } - - val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { + + val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + case None => MetricsSystem.timeUnits(CSV_DEFAULT_UNIT) } - - val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { + + val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { case Some(s) => s - case None => CsvSink.CSV_DEFAULT_DIR + case None => CSV_DEFAULT_DIR } - + val reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) @@ -31,21 +39,11 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si .build(new File(pollDir)) override def start() { - reporter.start(pollPeriod, pollUnit) + reporter.start(pollPeriod, pollUnit) } - + override def stop() { reporter.stop() } } -object CsvSink { - val CSV_KEY_PERIOD = "period" - val CSV_KEY_UNIT = "unit" - val CSV_KEY_DIR = "directory" - - val CSV_DEFAULT_PERIOD = "10" - val CSV_DEFAULT_UNIT = "second" - val CSV_DEFAULT_DIR = "/tmp/" -} - diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala index 57aa74512c..38158b8a2b 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -5,27 +5,26 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "DAGScheduler" - - metricRegistry.register(MetricRegistry.name("stage","failedStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.failed.size + metricRegistry.register(MetricRegistry.name("stage", "failedStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.failed.size }) - metricRegistry.register(MetricRegistry.name("stage","runningStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.running.size + metricRegistry.register(MetricRegistry.name("stage", "runningStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.running.size }) - metricRegistry.register(MetricRegistry.name("stage","waitingStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.waiting.size + metricRegistry.register(MetricRegistry.name("stage", "waitingStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.waiting.size }) - metricRegistry.register(MetricRegistry.name("job","allJobs"), new Gauge[Int] { - override def getValue: Int = dagScheduler.nextRunId.get() + metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.nextRunId.get() }) - metricRegistry.register(MetricRegistry.name("job","ActiveJobs"), new Gauge[Int] { - override def getValue: Int = dagScheduler.activeJobs.size + metricRegistry.register(MetricRegistry.name("job", "ActiveJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.activeJobs.size }) } diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index c0ce9259c8..f964827102 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -6,29 +6,29 @@ import spark.metrics.source.Source import spark.storage._ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory","maxMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "maxMem"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) maxMem } }) - metricRegistry.register(MetricRegistry.name("memory","remainingMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "remainingMem"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) remainingMem } }) - metricRegistry.register(MetricRegistry.name("disk","diskSpaceUsed"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) diskSpaceUsed } }) diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index f4c83cb644..bb1be4f4fc 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -9,60 +9,64 @@ import spark.metrics._ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + before { filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile() } test("MetricsConfig with default properties") { val conf = new MetricsConfig("dummy-file") + conf.initilize() + assert(conf.properties.size() === 2) assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") assert(conf.properties.getProperty("test-for-dummy") === null) - + val property = conf.getInstance("random") assert(property.size() === 2) assert(property.getProperty("sink.jmx.enabled") === "default") assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") } - + test("MetricsConfig with properties set") { val conf = new MetricsConfig(filePath) - + conf.initilize() + val masterProp = conf.getInstance("master") assert(masterProp.size() === 4) assert(masterProp.getProperty("sink.console.period") === "20") assert(masterProp.getProperty("sink.console.unit") === "minute") assert(masterProp.getProperty("sink.jmx.enabled") === "default") assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") - + val workerProp = conf.getInstance("worker") assert(workerProp.size() === 4) assert(workerProp.getProperty("sink.console.period") === "10") assert(workerProp.getProperty("sink.console.unit") === "second") } - + test("MetricsConfig with subProperties") { val conf = new MetricsConfig(filePath) - + conf.initilize() + val propCategories = conf.propertyCategories assert(propCategories.size === 2) - + val masterProp = conf.getInstance("master") - val sourceProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) + val sourceProps = conf.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) assert(sourceProps.size === 1) assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") - - val sinkProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SINK_REGEX) + + val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX) assert(sinkProps.size === 2) assert(sinkProps.contains("console")) assert(sinkProps.contains("jmx")) - + val consoleProps = sinkProps("console") assert(consoleProps.size() === 2) - + val jmxProps = sinkProps("jmx") assert(jmxProps.size() === 1) - } + } } diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 967be6ec47..f29bb9db67 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -9,32 +9,32 @@ import spark.metrics._ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf.file", filePath) } - + test("MetricsSystem with default config") { val metricsSystem = MetricsSystem.createMetricsSystem("default") val sources = metricsSystem.sources val sinks = metricsSystem.sinks - + assert(sources.length === 1) assert(sinks.length === 1) assert(sources(0).sourceName === "jvm") } - + test("MetricsSystem with sources add") { val metricsSystem = MetricsSystem.createMetricsSystem("test") val sources = metricsSystem.sources val sinks = metricsSystem.sinks - + assert(sources.length === 1) assert(sinks.length === 2) - + val source = new spark.deploy.master.MasterInstrumentation(null) metricsSystem.registerSource(source) assert(sources.length === 2) - } + } } From 5b4a2f2094258920613f50c0d337e1e92c08d012 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:31:53 +0800 Subject: [PATCH 279/419] Add metrics config template file --- conf/metrics.properties | 15 --------------- conf/metrics.properties.template | 14 ++++++++++++++ 2 files changed, 14 insertions(+), 15 deletions(-) delete mode 100644 conf/metrics.properties create mode 100644 conf/metrics.properties.template diff --git a/conf/metrics.properties b/conf/metrics.properties deleted file mode 100644 index 2ea3aaf345..0000000000 --- a/conf/metrics.properties +++ /dev/null @@ -1,15 +0,0 @@ -# syntax: [instance].[sink|source].[name].[options] - -*.sink.console.period=10 - -*.sink.console.unit=second - -*.sink.console.class=spark.metrics.sink.ConsoleSink - -master.sink.console.period=10 - -master.sink.console.unit=second - -worker.sink.console.period=20 - -worker.sink.console.unit=second diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template new file mode 100644 index 0000000000..b4204e1deb --- /dev/null +++ b/conf/metrics.properties.template @@ -0,0 +1,14 @@ +# syntax: [instance].[sink|source].[name].[options] + +#*.sink.console.period=10 + +#*.sink.console.unit=second + +#master.source.jvm.class=spark.metrics.source.JvmSource + +#worker.source.jvm.class=spark.metrics.source.JvmSource + +#driver.source.jvm.class=spark.metrics.source.JvmSource + +#executor.source.jvm.class=spark.metrics.source.JvmSource + From 05637de8423da85c5934cdfa8f07254133a58474 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 21:16:39 +0800 Subject: [PATCH 280/419] Change class xxxInstrumentation to class xxxSource --- core/src/main/scala/spark/deploy/master/Master.scala | 4 ++-- .../{MasterInstrumentation.scala => MasterSource.scala} | 2 +- core/src/main/scala/spark/deploy/worker/Worker.scala | 4 ++-- .../{WorkerInstrumentation.scala => WorkerSource.scala} | 2 +- core/src/main/scala/spark/executor/Executor.scala | 4 ++-- .../{ExecutorInstrumentation.scala => ExecutorSource.scala} | 2 +- core/src/test/scala/spark/metrics/MetricsSystemSuite.scala | 2 +- 7 files changed, 10 insertions(+), 10 deletions(-) rename core/src/main/scala/spark/deploy/master/{MasterInstrumentation.scala => MasterSource.scala} (90%) rename core/src/main/scala/spark/deploy/worker/{WorkerInstrumentation.scala => WorkerSource.scala} (93%) rename core/src/main/scala/spark/executor/{ExecutorInstrumentation.scala => ExecutorSource.scala} (94%) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 5f67366eb6..1d592206c0 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -58,7 +58,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val masterInstrumentation = new MasterInstrumentation(this) + val masterSource = new MasterSource(this) val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -77,7 +77,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - Master.metricsSystem.registerSource(masterInstrumentation) + Master.metricsSystem.registerSource(masterSource) Master.metricsSystem.start() } diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala similarity index 90% rename from core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala rename to core/src/main/scala/spark/deploy/master/MasterSource.scala index 4c3708cc4c..f94e5b2c34 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -private[spark] class MasterInstrumentation(val master: Master) extends Source { +private[spark] class MasterSource(val master: Master) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "master" diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index eaa1c1806f..5c0f77fd75 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -68,7 +68,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 - val workerInstrumentation = new WorkerInstrumentation(this) + val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -102,7 +102,7 @@ private[spark] class Worker( connectToMaster() startWebUi() - Worker.metricsSystem.registerSource(workerInstrumentation) + Worker.metricsSystem.registerSource(workerSource) Worker.metricsSystem.start() } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala similarity index 93% rename from core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala rename to core/src/main/scala/spark/deploy/worker/WorkerSource.scala index c76c0b4711..539eac71bd 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { +private[spark] class WorkerSource(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 4ea05dec1c..8a74a8d853 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -87,12 +87,12 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } ) - val executorInstrumentation = new ExecutorInstrumentation(this) + val executorSource = new ExecutorSource(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) SparkEnv.set(env) - env.metricsSystem.registerSource(executorInstrumentation) + env.metricsSystem.registerSource(executorSource) private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorSource.scala similarity index 94% rename from core/src/main/scala/spark/executor/ExecutorInstrumentation.scala rename to core/src/main/scala/spark/executor/ExecutorSource.scala index ad406f41b4..d8b531cb58 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Executor) extends Source { +class ExecutorSource(val executor: Executor) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "executor" diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index f29bb9db67..462c28e894 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -33,7 +33,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { assert(sources.length === 1) assert(sinks.length === 2) - val source = new spark.deploy.master.MasterInstrumentation(null) + val source = new spark.deploy.master.MasterSource(null) metricsSystem.registerSource(source) assert(sources.length === 2) } From 8d1ef7f2dfc453137b8dbbb72a7f1ca93b57bb15 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 4 Jul 2013 10:11:30 +0800 Subject: [PATCH 281/419] Code style changes --- conf/metrics.properties.template | 2 ++ .../src/main/scala/spark/deploy/master/MasterSource.scala | 4 ++-- core/src/main/scala/spark/metrics/MetricsConfig.scala | 8 ++++---- core/src/main/scala/spark/metrics/sink/ConsoleSink.scala | 1 - 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index b4204e1deb..c7e24aa36c 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,5 +1,7 @@ # syntax: [instance].[sink|source].[name].[options] +#*.sink.console.class=spark.metrics.sink.ConsoleSink + #*.sink.console.period=10 #*.sink.console.unit=second diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala index f94e5b2c34..65c22320d6 100644 --- a/core/src/main/scala/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -1,6 +1,6 @@ package spark.deploy.master -import com.codahale.metrics.{Gauge,MetricRegistry} +import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source @@ -10,7 +10,7 @@ private[spark] class MasterSource(val master: Master) extends Source { // Gauge for worker numbers in cluster metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { - override def getValue: Int = master.workers.size + override def getValue: Int = master.workers.size }) // Gauge for application numbers in cluster diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index b1f6a1e596..5066b7ac22 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -3,14 +3,14 @@ package spark.metrics import java.util.Properties import java.io.{File, FileInputStream} -import scala.collection.mutable.HashMap +import scala.collection.mutable import scala.util.matching.Regex private[spark] class MetricsConfig(val configFile: String) { val properties = new Properties() val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r - var propertyCategories: HashMap[String, Properties] = null + var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.jmx.enabled", "default") @@ -43,8 +43,8 @@ private[spark] class MetricsConfig(val configFile: String) { } } - def subProperties(prop: Properties, regex: Regex): HashMap[String, Properties] = { - val subProperties = new HashMap[String, Properties] + def subProperties(prop: Properties, regex: Regex): mutable.HashMap[String, Properties] = { + val subProperties = new mutable.HashMap[String, Properties] import scala.collection.JavaConversions._ prop.foreach { kv => if (regex.findPrefixOf(kv._1) != None) { diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index c67c0ee912..437f24a575 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,7 +8,6 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val CONSOLE_DEFAULT_PERIOD = "10" val CONSOLE_DEFAULT_UNIT = "second" From 31ec72b243459e0d1d9c89033328e1d79f3aa76f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 16 Jul 2013 10:57:19 +0800 Subject: [PATCH 282/419] Code refactor according to comments --- conf/metrics.properties.template | 73 ++++++++++++++++- .../scala/spark/deploy/master/Master.scala | 11 +-- .../spark/deploy/master/MasterSource.scala | 2 +- .../scala/spark/deploy/worker/Worker.scala | 11 +-- .../spark/deploy/worker/WorkerSource.scala | 12 +-- .../scala/spark/executor/ExecutorSource.scala | 8 +- .../scala/spark/metrics/MetricsConfig.scala | 50 +++++++----- .../scala/spark/metrics/MetricsSystem.scala | 81 ++++++++++++++----- .../spark/metrics/sink/ConsoleSink.scala | 12 +-- .../scala/spark/metrics/sink/CsvSink.scala | 12 +-- .../spark/scheduler/DAGSchedulerSource.scala | 10 +-- .../spark/storage/BlockManagerSource.scala | 31 ++++--- .../resources/test_metrics_config.properties | 6 +- .../resources/test_metrics_system.properties | 4 +- .../spark/metrics/MetricsConfigSuite.scala | 38 ++++----- .../spark/metrics/MetricsSystemSuite.scala | 13 ++- 16 files changed, 249 insertions(+), 125 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index c7e24aa36c..0486ca4c79 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,11 +1,82 @@ # syntax: [instance].[sink|source].[name].[options] +# "instance" specify "who" (the role) use metrics system. In spark there are +# several roles like master, worker, executor, driver, these roles will +# create metrics system for monitoring. So instance represents these roles. +# Currently in Spark, several instances have already implemented: master, +# worker, executor, driver. +# +# [instance] field can be "master", "worker", "executor", "driver", which means +# only the specified instance has this property. +# a wild card "*" can be used to represent instance name, which means all the +# instances will have this property. +# +# "source" specify "where" (source) to collect metrics data. In metrics system, +# there exists two kinds of source: +# 1. Spark internal source, like MasterSource, WorkerSource, etc, which will +# collect Spark component's internal state, these sources are related to +# instance and will be added after specific metrics system is created. +# 2. Common source, like JvmSource, which will collect low level state, is +# configured by configuration and loaded through reflection. +# +# "sink" specify "where" (destination) to output metrics data to. Several sinks +# can be coexisted and flush metrics to all these sinks. +# +# [sink|source] field specify this property is source related or sink, this +# field can only be source or sink. +# +# [name] field specify the name of source or sink, this is custom defined. +# +# [options] field is the specific property of this source or sink, this source +# or sink is responsible for parsing this property. +# +# Notes: +# 1. Sinks should be added through configuration, like console sink, class +# full name should be specified by class property. +# 2. Some sinks can specify polling period, like console sink, which is 10 seconds, +# it should be attention minimal polling period is 1 seconds, any period +# below than 1s is illegal. +# 3. Wild card property can be overlapped by specific instance property, for +# example, *.sink.console.period can be overlapped by master.sink.console.period. +# 4. A metrics specific configuration +# "spark.metrics.conf=${SPARK_HOME}/conf/metrics.properties" should be +# added to Java property using -Dspark.metrics.conf=xxx if you want to +# customize metrics system, or you can put it in ${SPARK_HOME}/conf, +# metrics system will search and load it automatically. + +# Enable JmxSink for all instances by class name +#*.sink.jmx.class=spark.metrics.sink.JmxSink + +# Enable ConsoleSink for all instances by class name #*.sink.console.class=spark.metrics.sink.ConsoleSink +# Polling period for ConsoleSink #*.sink.console.period=10 -#*.sink.console.unit=second +#*.sink.console.unit=seconds +# Master instance overlap polling period +#master.sink.console.period=15 + +#master.sink.console.unit=seconds + +# Enable CsvSink for all instances +#*.sink.csv.class=spark.metrics.sink.CsvSink + +# Polling period for CsvSink +#*.sink.csv.period=1 + +#*.sink.csv.unit=minutes + +# Polling directory for CsvSink +#*.sink.csv.directory=/tmp/ + +# Worker instance overlap polling period +#worker.sink.csv.period=10 + +#worker.sink.csv.unit=minutes + +# Enable jvm source for instance master, worker, driver and executor #master.source.jvm.class=spark.metrics.source.JvmSource #worker.source.jvm.class=spark.metrics.source.JvmSource diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 1d592206c0..9692af5295 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -58,6 +58,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") + val metricsSystem = MetricsSystem.createMetricsSystem("master") val masterSource = new MasterSource(this) val masterPublicAddress = { @@ -77,12 +78,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - Master.metricsSystem.registerSource(masterSource) - Master.metricsSystem.start() + metricsSystem.registerSource(masterSource) + metricsSystem.start() } override def postStop() { webUi.stop() + metricsSystem.stop() } override def receive = { @@ -322,17 +324,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } - - override def postStop() { - Master.metricsSystem.stop() - } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r - private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala index 65c22320d6..b8cfa6a773 100644 --- a/core/src/main/scala/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -19,7 +19,7 @@ private[spark] class MasterSource(val master: Master) extends Source { }) // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("waitingApps", "number"), new Gauge[Int] { override def getValue: Int = master.waitingApps.size }) } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 5c0f77fd75..8fa0d12b82 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -68,6 +68,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + val metricsSystem = MetricsSystem.createMetricsSystem("worker") val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed @@ -100,10 +101,9 @@ private[spark] class Worker( webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() - startWebUi() - Worker.metricsSystem.registerSource(workerSource) - Worker.metricsSystem.start() + metricsSystem.registerSource(workerSource) + metricsSystem.start() } def connectToMaster() { @@ -185,14 +185,11 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) webUi.stop() - - Worker.metricsSystem.stop() + metricsSystem.stop() } } private[spark] object Worker { - private val metricsSystem = MetricsSystem.createMetricsSystem("worker") - def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala index 539eac71bd..39cb8e5690 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala @@ -8,27 +8,27 @@ private[spark] class WorkerSource(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("executors", "number"), new Gauge[Int] { override def getValue: Int = worker.executors.size }) // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("coresUsed", "number"), new Gauge[Int] { override def getValue: Int = worker.coresUsed }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("memUsed", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryUsed }) // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("coresFree", "number"), new Gauge[Int] { override def getValue: Int = worker.coresFree }) - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { + // Gauge for memory free of this worker + metricRegistry.register(MetricRegistry.name("memFree", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryFree }) } diff --git a/core/src/main/scala/spark/executor/ExecutorSource.scala b/core/src/main/scala/spark/executor/ExecutorSource.scala index d8b531cb58..94116edfcf 100644 --- a/core/src/main/scala/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/spark/executor/ExecutorSource.scala @@ -9,22 +9,22 @@ class ExecutorSource(val executor: Executor) extends Source { val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts - metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "activeTask", "count"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed - metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("threadpool", "completeTask", "count"), new Gauge[Long] { override def getValue: Long = executor.threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads - metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "currentPool", "size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool - metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "maxPool", "size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) } diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 5066b7ac22..ed505b0aa7 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -1,44 +1,58 @@ package spark.metrics import java.util.Properties -import java.io.{File, FileInputStream} +import java.io.{File, FileInputStream, InputStream, IOException} import scala.collection.mutable import scala.util.matching.Regex -private[spark] class MetricsConfig(val configFile: String) { - val properties = new Properties() +import spark.Logging + +private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { + initLogging() + val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + val METRICS_CONF = "metrics.properties" + + val properties = new Properties() var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { - prop.setProperty("*.sink.jmx.enabled", "default") - prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + // empty function, any default property can be set here } - def initilize() { + def initialize() { //Add default properties in case there's no properties file setDefaultProperties(properties) - val confFile = new File(configFile) - if (confFile.exists()) { - var fis: FileInputStream = null - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() + // If spark.metrics.conf is not set, try to get file in class path + var is: InputStream = null + try { + is = configFile match { + case Some(f) => new FileInputStream(f) + case None => getClass.getClassLoader.getResourceAsStream(METRICS_CONF) } + + if (is != null) { + properties.load(is) + } + } catch { + case e: Exception => logError("Error loading configure file", e) + } finally { + if (is != null) is.close() } propertyCategories = subProperties(properties, INSTANCE_REGEX) if (propertyCategories.contains(DEFAULT_PREFIX)) { import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(DEFAULT_PREFIX) - for ((inst, prop) <- propertyCategories; p <- defaultProperty - if inst != DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { - prop.setProperty(p._1, p._2) + for { (inst, prop) <- propertyCategories + if (inst != DEFAULT_PREFIX) + (k, v) <- defaultProperty + if (prop.getProperty(k) == null) } { + prop.setProperty(k, v) } } } @@ -58,7 +72,7 @@ private[spark] class MetricsConfig(val configFile: String) { def getInstance(inst: String): Properties = { propertyCategories.get(inst) match { case Some(s) => s - case None => propertyCategories(DEFAULT_PREFIX) + case None => propertyCategories.getOrElse(DEFAULT_PREFIX, new Properties) } } } diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 54f6c6e4da..2f87577ff3 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -11,19 +11,51 @@ import spark.Logging import spark.metrics.sink.Sink import spark.metrics.source.Source +/** + * Spark Metrics System, created by specific "instance", combined by source, + * sink, periodically poll source metrics data to sink destinations. + * + * "instance" specify "who" (the role) use metrics system. In spark there are several roles + * like master, worker, executor, client driver, these roles will create metrics system + * for monitoring. So instance represents these roles. Currently in Spark, several instances + * have already implemented: master, worker, executor, driver. + * + * "source" specify "where" (source) to collect metrics data. In metrics system, there exists + * two kinds of source: + * 1. Spark internal source, like MasterSource, WorkerSource, etc, which will collect + * Spark component's internal state, these sources are related to instance and will be + * added after specific metrics system is created. + * 2. Common source, like JvmSource, which will collect low level state, is configured by + * configuration and loaded through reflection. + * + * "sink" specify "where" (destination) to output metrics data to. Several sinks can be + * coexisted and flush metrics to all these sinks. + * + * Metrics configuration format is like below: + * [instance].[sink|source].[name].[options] = xxxx + * + * [instance] can be "master", "worker", "executor", "driver", which means only the specified + * instance has this property. + * wild card "*" can be used to replace instance name, which means all the instances will have + * this property. + * + * [sink|source] means this property belongs to source or sink. This field can only be source or sink. + * + * [name] specify the name of sink or source, it is custom defined. + * + * [options] is the specific property of this source or sink. + */ private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") - val metricsConfig = new MetricsConfig(confFile) + val confFile = System.getProperty("spark.metrics.conf") + val metricsConfig = new MetricsConfig(Option(confFile)) val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] val registry = new MetricRegistry() - val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") - - metricsConfig.initilize() + metricsConfig.initialize() registerSources() registerSinks() @@ -37,7 +69,11 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - registry.register(source.sourceName, source.metricRegistry) + try { + registry.register(source.sourceName, source.metricRegistry) + } catch { + case e: IllegalArgumentException => logInfo("Metrics already registered", e) + } } def registerSources() { @@ -51,7 +87,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val source = Class.forName(classPath).newInstance() registerSource(source.asInstanceOf[Source]) } catch { - case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Source class " + classPath + " cannot be instantialized", e) } } } @@ -61,18 +97,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) sinkConfigs.foreach { kv => - val classPath = if (DEFAULT_SINKS.contains(kv._1)) { - DEFAULT_SINKS(kv._1) - } else { - // For non-default sink, a property class should be set and create using reflection - kv._2.getProperty("class") - } + val classPath = kv._2.getProperty("class") try { - val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) + val sink = Class.forName(classPath) + .getConstructor(classOf[Properties], classOf[MetricRegistry]) .newInstance(kv._2, registry) sinks += sink.asInstanceOf[Sink] } catch { - case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } } } @@ -81,12 +113,17 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin private[spark] object MetricsSystem { val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val timeUnits = Map( - "illisecond" -> TimeUnit.MILLISECONDS, - "second" -> TimeUnit.SECONDS, - "minute" -> TimeUnit.MINUTES, - "hour" -> TimeUnit.HOURS, - "day" -> TimeUnit.DAYS) - def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) + val MINIMAL_POLL_UNIT = TimeUnit.SECONDS + val MINIMAL_POLL_PERIOD = 1 + + def checkMinimalPollingPeriod(pollUnit: TimeUnit, pollPeriod: Int) { + val period = MINIMAL_POLL_UNIT.convert(pollPeriod, pollUnit) + if (period < MINIMAL_POLL_PERIOD) { + throw new IllegalArgumentException("Polling period " + pollPeriod + " " + pollUnit + + " below than minimal polling period ") + } + } + + def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) } diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 437f24a575..eaaac5d153 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,22 +8,24 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val CONSOLE_DEFAULT_PERIOD = "10" - val CONSOLE_DEFAULT_UNIT = "second" + val CONSOLE_DEFAULT_PERIOD = 10 + val CONSOLE_DEFAULT_UNIT = "SECONDS" val CONSOLE_KEY_PERIOD = "period" val CONSOLE_KEY_UNIT = "unit" val pollPeriod = Option(property.getProperty(CONSOLE_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CONSOLE_DEFAULT_PERIOD.toInt + case None => CONSOLE_DEFAULT_PERIOD } val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { - case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CONSOLE_DEFAULT_UNIT) + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(CONSOLE_DEFAULT_UNIT) } + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index a8ca819e87..aa5bff0d34 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -13,19 +13,21 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si val CSV_KEY_UNIT = "unit" val CSV_KEY_DIR = "directory" - val CSV_DEFAULT_PERIOD = "10" - val CSV_DEFAULT_UNIT = "second" + val CSV_DEFAULT_PERIOD = 10 + val CSV_DEFAULT_UNIT = "SECONDS" val CSV_DEFAULT_DIR = "/tmp/" val pollPeriod = Option(property.getProperty(CSV_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CSV_DEFAULT_PERIOD.toInt + case None => CSV_DEFAULT_PERIOD } val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { - case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CSV_DEFAULT_UNIT) + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { case Some(s) => s diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala index 38158b8a2b..87d27cc70d 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -8,23 +8,23 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends val metricRegistry = new MetricRegistry() val sourceName = "DAGScheduler" - metricRegistry.register(MetricRegistry.name("stage", "failedStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "failedStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.failed.size }) - metricRegistry.register(MetricRegistry.name("stage", "runningStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "runningStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.running.size }) - metricRegistry.register(MetricRegistry.name("stage", "waitingStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "waitingStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.waiting.size }) - metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("job", "allJobs", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.nextRunId.get() }) - metricRegistry.register(MetricRegistry.name("job", "ActiveJobs"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("job", "activeJobs", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.activeJobs.size }) } diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index f964827102..4faa715c94 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -9,27 +9,40 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends val metricRegistry = new MetricRegistry() val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory", "maxMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "maxMem", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - maxMem + val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + maxMem / 1024 / 1024 } }) - metricRegistry.register(MetricRegistry.name("memory", "remainingMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "remainingMem", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - remainingMem + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + remainingMem / 1024 / 1024 } }) - metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "memUsed", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) - diskSpaceUsed + val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + (maxMem - remainingMem) / 1024 / 1024 + } + }) + + metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed", "MBytes"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val diskSpaceUsed = storageStatusList + .flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_ + _) + .getOrElse(0L) + + diskSpaceUsed / 1024 / 1024 } }) } diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties index 2011940003..2b31ddf2eb 100644 --- a/core/src/test/resources/test_metrics_config.properties +++ b/core/src/test/resources/test_metrics_config.properties @@ -1,6 +1,6 @@ *.sink.console.period = 10 -*.sink.console.unit = second +*.sink.console.unit = seconds *.source.jvm.class = spark.metrics.source.JvmSource master.sink.console.period = 20 -master.sink.console.unit = minute - +master.sink.console.unit = minutes + diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties index 06afbc6625..d5479f0298 100644 --- a/core/src/test/resources/test_metrics_system.properties +++ b/core/src/test/resources/test_metrics_system.properties @@ -1,7 +1,7 @@ *.sink.console.period = 10 -*.sink.console.unit = second +*.sink.console.unit = seconds test.sink.console.class = spark.metrics.sink.ConsoleSink test.sink.dummy.class = spark.metrics.sink.DummySink test.source.dummy.class = spark.metrics.source.DummySource test.sink.console.period = 20 -test.sink.console.unit = minute +test.sink.console.unit = minutes diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index bb1be4f4fc..87cd2ffad2 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -15,40 +15,36 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { } test("MetricsConfig with default properties") { - val conf = new MetricsConfig("dummy-file") - conf.initilize() + val conf = new MetricsConfig(Option("dummy-file")) + conf.initialize() - assert(conf.properties.size() === 2) - assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") - assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") + assert(conf.properties.size() === 0) assert(conf.properties.getProperty("test-for-dummy") === null) val property = conf.getInstance("random") - assert(property.size() === 2) - assert(property.getProperty("sink.jmx.enabled") === "default") - assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") + assert(property.size() === 0) } test("MetricsConfig with properties set") { - val conf = new MetricsConfig(filePath) - conf.initilize() + val conf = new MetricsConfig(Option(filePath)) + conf.initialize() val masterProp = conf.getInstance("master") - assert(masterProp.size() === 4) + assert(masterProp.size() === 3) assert(masterProp.getProperty("sink.console.period") === "20") - assert(masterProp.getProperty("sink.console.unit") === "minute") - assert(masterProp.getProperty("sink.jmx.enabled") === "default") - assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") + assert(masterProp.getProperty("sink.console.unit") === "minutes") + assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") val workerProp = conf.getInstance("worker") - assert(workerProp.size() === 4) + assert(workerProp.size() === 3) assert(workerProp.getProperty("sink.console.period") === "10") - assert(workerProp.getProperty("sink.console.unit") === "second") + assert(workerProp.getProperty("sink.console.unit") === "seconds") + assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") } test("MetricsConfig with subProperties") { - val conf = new MetricsConfig(filePath) - conf.initilize() + val conf = new MetricsConfig(Option(filePath)) + conf.initialize() val propCategories = conf.propertyCategories assert(propCategories.size === 2) @@ -59,14 +55,10 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX) - assert(sinkProps.size === 2) + assert(sinkProps.size === 1) assert(sinkProps.contains("console")) - assert(sinkProps.contains("jmx")) val consoleProps = sinkProps("console") assert(consoleProps.size() === 2) - - val jmxProps = sinkProps("jmx") - assert(jmxProps.size() === 1) } } diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 462c28e894..c189996417 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -12,7 +12,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() - System.setProperty("spark.metrics.conf.file", filePath) + System.setProperty("spark.metrics.conf", filePath) } test("MetricsSystem with default config") { @@ -20,9 +20,8 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val sources = metricsSystem.sources val sinks = metricsSystem.sinks - assert(sources.length === 1) - assert(sinks.length === 1) - assert(sources(0).sourceName === "jvm") + assert(sources.length === 0) + assert(sinks.length === 0) } test("MetricsSystem with sources add") { @@ -30,11 +29,11 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val sources = metricsSystem.sources val sinks = metricsSystem.sinks - assert(sources.length === 1) - assert(sinks.length === 2) + assert(sources.length === 0) + assert(sinks.length === 1) val source = new spark.deploy.master.MasterSource(null) metricsSystem.registerSource(source) - assert(sources.length === 2) + assert(sources.length === 1) } } From 93c6015f82d4d27a6f09686b1e849be1cbbd0615 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 10:53:02 -0700 Subject: [PATCH 283/419] Shows task status and running tasks on Stage Page: fixes SPARK-804 and 811 --- .../spark/scheduler/cluster/TaskInfo.scala | 12 ++++++++++++ .../scala/spark/ui/exec/ExecutorsUI.scala | 5 +++++ .../scala/spark/ui/jobs/JobProgressUI.scala | 16 ++++++++++++++++ .../main/scala/spark/ui/jobs/StagePage.scala | 19 +++++++++++++------ 4 files changed, 46 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index a1ebd48b01..f840100eca 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -51,6 +51,18 @@ class TaskInfo( def running: Boolean = !finished + def status: String = { + if (running) + return "RUNNING" + if (failed) + return "FAILED" + if (successful) + return "SUCCESSFUL" + if (finished) + return "FINISHED" + "UNKNOWN" + } + def duration: Long = { if (!finished) { throw new UnsupportedOperationException("duration() called on unfinished tasks") diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index bad5c442ab..db1c902955 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -124,6 +124,10 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { if (!executorToTasksActive.contains(eid)) executorToTasksActive(eid) = HashSet[Long]() executorToTasksActive(eid) += taskStart.taskInfo.taskId + val taskList = executorToTaskInfos.getOrElse( + eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskStart.taskInfo, None, None)) + executorToTaskInfos(eid) = taskList } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { @@ -142,6 +146,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList -= ((taskEnd.taskInfo, None, None)) taskList += ((taskEnd.taskInfo, metrics, failureInfo)) executorToTaskInfos(eid) = taskList } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 44dcf82d11..6e332415db 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -65,6 +65,7 @@ private[spark] class JobProgressListener extends SparkListener { val completedStages = ListBuffer[Stage]() val failedStages = ListBuffer[Stage]() + val stageToTasksActive = HashMap[Int, HashSet[Long]]() val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = @@ -93,8 +94,22 @@ private[spark] class JobProgressListener extends SparkListener { override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = activeStages += stageSubmitted.stage + override def onTaskStart(taskStart: SparkListenerTaskStart) { + val sid = taskStart.task.stageId + if (!stageToTasksActive.contains(sid)) + stageToTasksActive(sid) = HashSet[Long]() + stageToTasksActive(sid) += taskStart.taskInfo.taskId + val taskList = stageToTaskInfos.getOrElse( + sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskStart.taskInfo, None, None)) + stageToTaskInfos(sid) = taskList + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId + if (!stageToTasksActive.contains(sid)) + stageToTasksActive(sid) = HashSet[Long]() + stageToTasksActive(sid) -= taskEnd.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -106,6 +121,7 @@ private[spark] class JobProgressListener extends SparkListener { } val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList -= ((taskEnd.taskInfo, None, None)) taskList += ((taskEnd.taskInfo, metrics, failureInfo)) stageToTaskInfos(sid) = taskList } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 292966f23a..b77b29045e 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -41,8 +41,8 @@ private[spark] class StagePage(parent: JobProgressUI) { if (!listener.stageToTaskInfos.contains(stageId)) { val content =
        -

        Summary Metrics

        No tasks have finished yet -

        Tasks

        No tasks have finished yet +

        Summary Metrics

        No tasks have started yet +

        Tasks

        No tasks have started yet
        return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } @@ -53,7 +53,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleWrite = listener.hasShuffleWrite(stageId) val taskHeaders: Seq[String] = - Seq("Task ID", "Duration", "Locality Level", "Worker", "Launch Time") ++ + Seq("Task ID", "Status", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ {if (shuffleWrite) Seq("Shuffle Write") else Nil} ++ Seq("Details") @@ -61,7 +61,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskTable = listingTable(taskHeaders, taskRow, tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => Option(t._2).isDefined) + val validTasks = tasks.filter(t => t._1.status == "SUCCESSFUL" && (Option(t._2).isDefined)) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { @@ -108,10 +108,17 @@ private[spark] class StagePage(parent: JobProgressUI) { def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = trace.map(e => {e.toString}) val (info, metrics, exception) = taskData + + val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) + else metrics.map(m => m.executorRunTime).getOrElse(1) + val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) + else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + {info.taskId} - m.executorRunTime.toString}.getOrElse("1")}> - {metrics.map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} + {info.status} + + {formatDuration} {info.taskLocality} {info.hostPort} From bd3931c87415679fef8b91390d50188c3da98430 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 11:27:17 -0700 Subject: [PATCH 284/419] Changed ifs with returns to if/else --- .../spark/scheduler/cluster/TaskInfo.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index f840100eca..afcfe96a09 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -53,14 +53,15 @@ class TaskInfo( def status: String = { if (running) - return "RUNNING" - if (failed) - return "FAILED" - if (successful) - return "SUCCESSFUL" - if (finished) - return "FINISHED" - "UNKNOWN" + "RUNNING" + else if (failed) + "FAILED" + else if (successful) + "SUCCESSFUL" + else if (finished) + "FINISHED" + else + "UNKNOWN" } def duration: Long = { From 4280e1768d09e43b55f26556e14a8eb70d9b9fad Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 12:48:48 -0700 Subject: [PATCH 285/419] Removed finished status for task info, changed name of success case --- core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index afcfe96a09..c693b722ac 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -57,9 +57,7 @@ class TaskInfo( else if (failed) "FAILED" else if (successful) - "SUCCESSFUL" - else if (finished) - "FINISHED" + "SUCCESS" else "UNKNOWN" } From 57009eef9099aa3b35b9262d9637af3dc94266a1 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 13:43:09 -0700 Subject: [PATCH 286/419] Fixed consistency of "success" status string --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index b77b29045e..654f347723 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -61,7 +61,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskTable = listingTable(taskHeaders, taskRow, tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => t._1.status == "SUCCESSFUL" && (Option(t._2).isDefined)) + val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && (Option(t._2).isDefined)) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { From 1d101928069c31aec039bb8b69c9bf4c204eead4 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 22 Jul 2013 13:11:35 -0700 Subject: [PATCH 287/419] Fix setting of SPARK_EXAMPLES_JAR --- bin/compute-classpath.sh | 11 ----------- run | 13 +++++++++++++ 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index eb270a5428..e4ce1ca848 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -76,17 +76,6 @@ function dev_classpath { CLASSPATH="$CLASSPATH:$jar" done - # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack - # to avoid the -sources and -doc packages that are built by publish-local. - if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then - # Use the JAR from the SBT build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` - fi - if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then - # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` - fi - # Add Scala standard library if [ -z "$SCALA_LIBRARY_PATH" ]; then if [ -z "$SCALA_HOME" ]; then diff --git a/run b/run index f49aa92796..4cffda4708 100755 --- a/run +++ b/run @@ -17,6 +17,8 @@ # limitations under the License. # +SCALA_VERSION=2.9.3 + # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" @@ -136,6 +138,17 @@ if [ ! -f "$FWDIR/RELEASE" ]; then echo "You need to compile Spark repl module before running this program" >&2 exit 1 fi + + # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack + # to avoid the -sources and -doc packages that are built by publish-local. + if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` + fi + if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the Maven build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + fi fi # Compute classpath using external script From eef678703eed96544224209b1555618968b2eb3f Mon Sep 17 00:00:00 2001 From: Xinghao Date: Wed, 24 Jul 2013 15:32:50 -0700 Subject: [PATCH 288/419] Adding SVM and Lasso, moving LogisticRegression to classification from regression Also, add regularization parameter to SGD --- .../mllib/classification/Classification.scala | 21 +++ .../LogisticRegression.scala | 28 +-- .../LogisticRegressionGenerator.scala | 2 +- .../spark/mllib/classification/SVM.scala | 170 ++++++++++++++++++ .../mllib/classification/SVMGenerator.scala | 45 +++++ .../spark/mllib/optimization/Gradient.scala | 28 +++ .../mllib/optimization/GradientDescent.scala | 10 +- .../spark/mllib/optimization/Updater.scala | 31 +++- .../scala/spark/mllib/regression/Lasso.scala | 167 +++++++++++++++++ .../mllib/regression/LassoGenerator.scala | 44 +++++ .../LogisticRegressionSuite.scala | 2 +- .../spark/mllib/classification/SVMSuite.scala | 61 +++++++ .../spark/mllib/regression/LassoSuite.scala | 51 ++++++ 13 files changed, 642 insertions(+), 18 deletions(-) create mode 100644 mllib/src/main/scala/spark/mllib/classification/Classification.scala rename mllib/src/main/scala/spark/mllib/{regression => classification}/LogisticRegression.scala (84%) rename mllib/src/main/scala/spark/mllib/{regression => classification}/LogisticRegressionGenerator.scala (96%) create mode 100644 mllib/src/main/scala/spark/mllib/classification/SVM.scala create mode 100644 mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/Lasso.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala rename mllib/src/test/scala/spark/mllib/{regression => classification}/LogisticRegressionSuite.scala (97%) create mode 100644 mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala create mode 100644 mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/Classification.scala new file mode 100644 index 0000000000..7f1eb21079 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/classification/Classification.scala @@ -0,0 +1,21 @@ +package spark.mllib.classification + +import spark.RDD + +trait ClassificationModel { + /** + * Predict values for the given data set using the model trained. + * + * @param testData RDD representing data points to be predicted + * @return RDD[Double] where each entry contains the corresponding prediction + */ + def predict(testData: RDD[Array[Double]]): RDD[Double] + + /** + * Predict values for a single data point using the model trained. + * + * @param testData array representing a single data point + * @return Double prediction from the trained model + */ + def predict(testData: Array[Double]): Double +} diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala similarity index 84% rename from mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala rename to mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index e4db7bb9b7..f39c1ec52e 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -1,4 +1,4 @@ -package spark.mllib.regression +package spark.mllib.classification import spark.{Logging, RDD, SparkContext} import spark.mllib.optimization._ @@ -13,7 +13,7 @@ import org.jblas.DoubleMatrix class LogisticRegressionModel( val weights: DoubleMatrix, val intercept: Double, - val losses: Array[Double]) extends RegressionModel { + val losses: Array[Double]) extends ClassificationModel { override def predict(testData: spark.RDD[Array[Double]]) = { testData.map { x => @@ -29,14 +29,14 @@ class LogisticRegressionModel( } } -class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, +class LogisticRegression private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 1.0, 100) + def this() = this(1.0, 1.0, 1.0, 100) /** * Set the step size per-iteration of SGD. Default 1.0. @@ -69,7 +69,7 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, regParam, miniBatchFraction) val weightsScaled = weights.getRange(1, weights.length) val intercept = weights.get(0) @@ -96,16 +96,18 @@ object LogisticRegression { * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, stepSize: Double, + regParam: Double, miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(input) + new LogisticRegression(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -115,16 +117,18 @@ object LogisticRegression { * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. * @param numIterations Number of iterations of gradient descent to run. * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, - stepSize: Double) + stepSize: Double, + regParam: Double) : LogisticRegressionModel = { - train(input, numIterations, stepSize, 1.0) + train(input, numIterations, stepSize, regParam, 1.0) } /** @@ -141,17 +145,17 @@ object LogisticRegression { numIterations: Int) : LogisticRegressionModel = { - train(input, numIterations, 1.0, 1.0) + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { - if (args.length != 4) { - println("Usage: LogisticRegression ") + if (args.length != 5) { + println("Usage: LogisticRegression ") System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) + val model = LogisticRegression.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala index 6e7c023bac..cde1148adf 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala @@ -1,4 +1,4 @@ -package spark.mllib.regression +package spark.mllib.classification import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala new file mode 100644 index 0000000000..aceb903f1d --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -0,0 +1,170 @@ +package spark.mllib.classification + +import scala.math.signum +import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * SVM using Stochastic Gradient Descent. + */ +class SVMModel( + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends ClassificationModel { + + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => { + println("Predicting " + x) + signum(new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept) + } + } + } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + signum(dataMat.dot(this.weights) + this.intercept) + } +} + +class SVM private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, + var numIters: Int) + extends Logging { + + /** + * Construct a SVM object with default parameters + */ + def this() = this(1.0, 1.0, 1.0, 100) + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + this.stepSize = step + this + } + + /** + * Set the regularization parameter. Default 1.0. + */ + def setRegParam(param: Double) = { + this.regParam = param + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + this.numIters = iters + this + } + + def train(input: RDD[(Double, Array[Double])]): SVMModel = { + // Add a extra variable consisting of all 1.0's for the intercept. + val data = input.map { case (y, features) => + (y, Array(1.0, features:_*)) + } + + val (weights, losses) = GradientDescent.runMiniBatchSGD( + data, new HingeGradient(), new SquaredL2Updater(), stepSize, numIters, regParam, miniBatchFraction) + + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) + + val model = new SVMModel(weightsScaled, intercept, losses) + + logInfo("Final model weights " + model.weights) + logInfo("Final model intercept " + model.intercept) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + model + } +} + +/** + * Top-level methods for calling SVM. + */ +object SVM { + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : SVMModel = + { + new SVM(stepSize, regParam, miniBatchFraction, numIterations).train(input) + } + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. + * @return a SVMModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double) + : SVMModel = + { + train(input, numIterations, stepSize, regParam, 1.0) + } + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a SVMModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int) + : SVMModel = + { + train(input, numIterations, 1.0, 0.10, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: SVM ") + System.exit(1) + } + val sc = new SparkContext(args(0), "SVM") + val data = MLUtils.loadLabeledData(sc, args(1)) + val model = SVM.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala new file mode 100644 index 0000000000..a5e2837343 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala @@ -0,0 +1,45 @@ +package spark.mllib.classification + +import scala.util.Random +import scala.math.signum + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.util.MLUtils + +object LassoGenerator { + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LassoGenerator " + + " ") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 + val parts: Int = if (args.length > 4) args(4).toInt else 2 + val eps = 3 + + val sc = new SparkContext(sparkMaster, "LassoGenerator") + + val globalRnd = new Random(94720) + val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => + val rnd = new Random(42 + idx) + + val x = Array.fill[Double](nfeatures) { + rnd.nextDouble() * 2.0 - 1.0 + } + val y = signum(((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1) + (y, x) + } + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 90b0999a5e..6ffc3b128b 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -31,3 +31,31 @@ class LogisticGradient extends Gradient { (gradient, loss) } } + + +class SquaredGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + val diff: Double = data.dot(weights) - label + + val loss = 0.5 * diff * diff + val gradient = data.mul(diff) + + (gradient, loss) + } +} + + +class HingeGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + + val dotProduct = data.dot(weights) + + if (1.0 > label * dotProduct) + (data.mul(-label), 1.0 - label * dotProduct) + else + (DoubleMatrix.zeros(1,weights.length), 0.0) + } +} + diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index eff853f379..bd8489c386 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -19,6 +19,7 @@ object GradientDescent { * @param updater - Updater object that will be used to update the model. * @param stepSize - stepSize to be used during update. * @param numIters - number of iterations that SGD should be run. + * @param regParam - regularization parameter * @param miniBatchFraction - fraction of the input data set that should be used for * one iteration of SGD. Default value 1.0. * @@ -31,6 +32,7 @@ object GradientDescent { updater: Updater, stepSize: Double, numIters: Int, + regParam: Double, miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { val lossHistory = new ArrayBuffer[Double](numIters) @@ -51,10 +53,14 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - lossHistory.append(lossSum / miniBatchSize + reg_val) - val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) + val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) weights = update._1 reg_val = update._2 + lossHistory.append(lossSum / miniBatchSize + reg_val) + /*** + Xinghao: The loss here is sum of lossSum computed using the weights before applying updater, + and reg_val using weights after applying updater + ***/ } (weights, lossHistory.toArray) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index ea80bfcbfd..64c54dfb0d 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -1,5 +1,6 @@ package spark.mllib.optimization +import scala.math._ import org.jblas.DoubleMatrix abstract class Updater extends Serializable { @@ -10,18 +11,44 @@ abstract class Updater extends Serializable { * @param gradient - Column matrix of size nx1 where n is the number of features. * @param stepSize - step size across iterations * @param iter - Iteration number + * @param regParam - Regularization parameter * * @return weightsNew - Column matrix containing updated weights * @return reg_val - regularization value */ - def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) } class SimpleUpdater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int): (DoubleMatrix, Double) = { + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { val normGradient = gradient.mul(stepSize / math.sqrt(iter)) (weightsOld.sub(normGradient), 0) } } + +class L1Updater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) + val newWeights = weightsOld.sub(normGradient) + (0 until newWeights.length).foreach(i => { + val wi = newWeights.get(i) + newWeights.put(i, signum(wi) * max(0.0, abs(wi) - regParam * thisIterStepSize)) + }) + (newWeights, newWeights.norm1 * regParam) + } +} + +class SquaredL2Updater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) + val newWeights = weightsOld.sub(normGradient).div(2.0 * thisIterStepSize * regParam + 1.0) + (newWeights, pow(newWeights.norm2,2.0) * regParam) + } +} + diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala new file mode 100644 index 0000000000..de410711a2 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -0,0 +1,167 @@ +package spark.mllib.regression + +import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Lasso using Stochastic Gradient Descent. + */ +class LassoModel( + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends RegressionModel { + + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => + new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept + } + } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + dataMat.dot(this.weights) + this.intercept + } +} + +class Lasso private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, + var numIters: Int) + extends Logging { + + /** + * Construct a Lasso object with default parameters + */ + def this() = this(1.0, 1.0, 1.0, 100) + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + this.stepSize = step + this + } + + /** + * Set the regularization parameter. Default 1.0. + */ + def setRegParam(param: Double) = { + this.regParam = param + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + this.numIters = iters + this + } + + def train(input: RDD[(Double, Array[Double])]): LassoModel = { + // Add a extra variable consisting of all 1.0's for the intercept. + val data = input.map { case (y, features) => + (y, Array(1.0, features:_*)) + } + + val (weights, losses) = GradientDescent.runMiniBatchSGD( + data, new SquaredGradient(), new L1Updater(), stepSize, numIters, regParam, miniBatchFraction) + + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) + + val model = new LassoModel(weightsScaled, intercept, losses) + + logInfo("Final model weights " + model.weights) + logInfo("Final model intercept " + model.intercept) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + model + } +} + +/** + * Top-level methods for calling Lasso. + */ +object Lasso { + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : LassoModel = + { + new Lasso(stepSize, regParam, miniBatchFraction, numIterations).train(input) + } + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LassoModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double) + : LassoModel = + { + train(input, numIterations, stepSize, regParam, 1.0) + } + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LassoModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int) + : LassoModel = + { + train(input, numIterations, 1.0, 0.10, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: Lasso ") + System.exit(1) + } + val sc = new SparkContext(args(0), "Lasso") + val data = MLUtils.loadLabeledData(sc, args(1)) + val model = Lasso.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala new file mode 100644 index 0000000000..d2d3bb33c7 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala @@ -0,0 +1,44 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.util.MLUtils + +object LassoGenerator { + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LassoGenerator " + + " ") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 + val parts: Int = if (args.length > 4) args(4).toInt else 2 + val eps = 3 + + val sc = new SparkContext(sparkMaster, "LassoGenerator") + + val globalRnd = new Random(94720) + val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => + val rnd = new Random(42 + idx) + + val x = Array.fill[Double](nfeatures) { + rnd.nextDouble() * 2.0 - 1.0 + } + val y = ((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1 + (y, x) + } + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala similarity index 97% rename from mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala rename to mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 04d3400cb4..13612e9a4a 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.mllib.regression +package spark.mllib.classification import scala.util.Random diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala new file mode 100644 index 0000000000..e3a6681ab2 --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -0,0 +1,61 @@ +package spark.mllib.classification + +import scala.util.Random +import scala.math.signum + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + +import java.io._ + +class SVMSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + test("SVM") { + val nPoints = 10000 + val rnd = new Random(42) + + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val A = 2.0 + val B = -1.5 + val C = 1.0 + + val y = (0 until nPoints).map { i => + signum(A + B * x1(i) + C * x2(i) + 0.0*rnd.nextGaussian()) + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val writer_data = new PrintWriter(new File("svmtest.dat")) + testData.foreach(yx => { + writer_data.write(yx._1 + "") + yx._2.foreach(xi => writer_data.write("\t" + xi)) + writer_data.write("\n")}) + writer_data.close() + + val svm = new SVM().setStepSize(1.0) + .setRegParam(1.0) + .setNumIterations(100) + + val model = svm.train(testRDD) + + val yPredict = (0 until nPoints).map(i => model.predict(Array(x1(i),x2(i)))) + + val accuracy = ((y zip yPredict).map(yy => if (yy._1==yy._2) 1 else 0).reduceLeft(_+_).toDouble / nPoints.toDouble) + + assert(accuracy >= 0.90, "Accuracy (" + accuracy + ") too low") + } +} diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala new file mode 100644 index 0000000000..90fedb3e84 --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -0,0 +1,51 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + + +class LassoSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + test("Lasso") { + val nPoints = 10000 + val rnd = new Random(42) + + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val A = 2.0 + val B = -1.5 + val C = 1.0e-2 + + val y = (0 until nPoints).map { i => + A + B * x1(i) + C * x2(i) + 0.1*rnd.nextGaussian() + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val ls = new Lasso().setStepSize(1.0) + .setRegParam(0.01) + .setNumIterations(20) + + val model = ls.train(testRDD) + + val weight0 = model.weights.get(0) + val weight1 = model.weights.get(1) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + } +} From c258718606a2960649dde0a4925fcf385d617c37 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 24 Jul 2013 16:12:32 -0700 Subject: [PATCH 289/419] Fix Maven build errors after previous commits --- pom.xml | 18 ++++--- tools/pom.xml | 133 ++++++++++++++++++++++++++++++++++++++++++++------ 2 files changed, 129 insertions(+), 22 deletions(-) diff --git a/pom.xml b/pom.xml index 1fe288b92c..4cf9589b07 100644 --- a/pom.xml +++ b/pom.xml @@ -248,6 +248,16 @@ lift-json_2.9.2 2.5
        + + com.codahale.metrics + metrics-core + 3.0.0 + + + com.codahale.metrics + metrics-jvm + 3.0.0 + org.scala-lang scala-compiler @@ -267,14 +277,6 @@ org.scala-lang scalap ${scala.version} - com.codahale.metrics - metrics-core - 3.0.0 - - - com.codahale.metrics - metrics-jvm - 3.0.0 diff --git a/tools/pom.xml b/tools/pom.xml index 3dfba5808c..1125aba4f1 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,28 +20,16 @@ org.spark-project spark-parent - 0.7.4-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml org.spark-project - spark-tools_2.9.3 + spark-tools jar Spark Project Tools http://spark-project.org/ - - - org.spark-project - spark-core_2.9.3 - ${project.version} - - - org.spark-project - spark-streaming_2.9.3 - ${project.version} - - target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes @@ -60,4 +48,121 @@ + + + + hadoop1 + + + org.spark-project + spark-core + ${project.version} + hadoop1 + + + org.spark-project + spark-streaming + ${project.version} + hadoop1 + + + org.apache.hadoop + hadoop-core + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop1 + + + + + + + hadoop2 + + + org.spark-project + spark-core + ${project.version} + hadoop2 + + + org.spark-project + spark-streaming + ${project.version} + hadoop2 + + + org.apache.hadoop + hadoop-core + provided + + + org.apache.hadoop + hadoop-client + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2 + + + + + + + hadoop2-yarn + + + org.spark-project + spark-core + ${project.version} + hadoop2-yarn + + + org.spark-project + spark-streaming + ${project.version} + hadoop2-yarn + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2-yarn + + + + + + From 8e0939f5a9c0823d51d0e94185b55d4a02628a53 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:43:57 -0700 Subject: [PATCH 290/419] refactor Kryo serializer support to use chill/chill-java --- .../src/main/scala/spark/KryoSerializer.scala | 145 ++++-------------- .../scala/spark/KryoSerializerSuite.scala | 32 +--- project/SparkBuild.scala | 5 +- 3 files changed, 38 insertions(+), 144 deletions(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index ee37da7948..2ea90d41c3 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -19,24 +19,15 @@ package spark import java.io._ import java.nio.ByteBuffer -import java.nio.channels.Channels - -import scala.collection.immutable -import scala.collection.mutable - -import com.esotericsoftware.kryo._ -import com.esotericsoftware.kryo.{Serializer => KSerializer} +import com.esotericsoftware.kryo.{Kryo, KryoException} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} -import de.javakaffee.kryoserializers.KryoReflectionFactorySupport - +import com.twitter.chill.ScalaKryoInstantiator import serializer.{SerializerInstance, DeserializationStream, SerializationStream} import spark.broadcast._ import spark.storage._ -private[spark] -class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { - +private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) def writeObject[T](t: T): SerializationStream = { @@ -48,9 +39,7 @@ class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends Seria def close() { output.close() } } -private[spark] -class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { - +private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { val input = new KryoInput(inStream) def readObject[T](): T = { @@ -58,7 +47,7 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser kryo.readClassAndObject(input).asInstanceOf[T] } catch { // DeserializationStream uses the EOF exception to indicate stopping condition. - case e: com.esotericsoftware.kryo.KryoException => throw new java.io.EOFException + case _: KryoException => throw new EOFException } } @@ -69,10 +58,9 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser } private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - - val kryo = ks.kryo.get() - val output = ks.output.get() - val input = ks.input.get() + val kryo = ks.newKryo() + val output = ks.newKryoOutput() + val input = ks.newKryoInput() def serialize[T](t: T): ByteBuffer = { output.clear() @@ -108,126 +96,52 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ * serialization. */ trait KryoRegistrator { - def registerClasses(kryo: Kryo): Unit + def registerClasses(kryo: Kryo) } /** * A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]]. */ class KryoSerializer extends spark.serializer.Serializer with Logging { + private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 - val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + def newKryoOutput() = new KryoOutput(bufferSize) - val kryo = new ThreadLocal[Kryo] { - override def initialValue = createKryo() - } + def newKryoInput() = new KryoInput(bufferSize) - val output = new ThreadLocal[KryoOutput] { - override def initialValue = new KryoOutput(bufferSize) - } - - val input = new ThreadLocal[KryoInput] { - override def initialValue = new KryoInput(bufferSize) - } - - def createKryo(): Kryo = { - val kryo = new KryoReflectionFactorySupport() + def newKryo(): Kryo = { + val instantiator = new ScalaKryoInstantiator + val kryo = instantiator.newKryo() + val classLoader = Thread.currentThread.getContextClassLoader // Register some commonly used classes val toRegister: Seq[AnyRef] = Seq( - // Arrays - Array(1), Array(1.0), Array(1.0f), Array(1L), Array(""), Array(("", "")), - Array(new java.lang.Object), Array(1.toByte), Array(true), Array('c'), - // Specialized Tuple2s - ("", ""), ("", 1), (1, 1), (1.0, 1.0), (1L, 1L), - (1, 1.0), (1.0, 1), (1L, 1.0), (1.0, 1L), (1, 1L), (1L, 1), - // Scala collections - List(1), mutable.ArrayBuffer(1), - // Options and Either - Some(1), Left(1), Right(1), - // Higher-dimensional tuples - (1, 1, 1), (1, 1, 1, 1), (1, 1, 1, 1, 1), - None, ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY, PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY), GotBlock("1", ByteBuffer.allocate(1)), GetBlock("1") ) - for (obj <- toRegister) { - kryo.register(obj.getClass) - } + + for (obj <- toRegister) kryo.register(obj.getClass) // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) - // Register some commonly used Scala singleton objects. Because these - // are singletons, we must return the exact same local object when we - // deserialize rather than returning a clone as FieldSerializer would. - class SingletonSerializer[T](obj: T) extends KSerializer[T] { - override def write(kryo: Kryo, output: KryoOutput, obj: T) {} - override def read(kryo: Kryo, input: KryoInput, cls: java.lang.Class[T]): T = obj - } - kryo.register(None.getClass, new SingletonSerializer[AnyRef](None)) - kryo.register(Nil.getClass, new SingletonSerializer[AnyRef](Nil)) - - // Register maps with a special serializer since they have complex internal structure - class ScalaMapSerializer(buildMap: Array[(Any, Any)] => scala.collection.Map[Any, Any]) - extends KSerializer[Array[(Any, Any)] => scala.collection.Map[Any, Any]] { - - //hack, look at https://groups.google.com/forum/#!msg/kryo-users/Eu5V4bxCfws/k-8UQ22y59AJ - private final val FAKE_REFERENCE = new Object() - override def write( - kryo: Kryo, - output: KryoOutput, - obj: Array[(Any, Any)] => scala.collection.Map[Any, Any]) { - val map = obj.asInstanceOf[scala.collection.Map[Any, Any]] - output.writeInt(map.size) - for ((k, v) <- map) { - kryo.writeClassAndObject(output, k) - kryo.writeClassAndObject(output, v) - } - } - override def read ( - kryo: Kryo, - input: KryoInput, - cls: Class[Array[(Any, Any)] => scala.collection.Map[Any, Any]]) - : Array[(Any, Any)] => scala.collection.Map[Any, Any] = { - kryo.reference(FAKE_REFERENCE) - val size = input.readInt() - val elems = new Array[(Any, Any)](size) - for (i <- 0 until size) { - val k = kryo.readClassAndObject(input) - val v = kryo.readClassAndObject(input) - elems(i)=(k,v) - } - buildMap(elems).asInstanceOf[Array[(Any, Any)] => scala.collection.Map[Any, Any]] - } - } - kryo.register(mutable.HashMap().getClass, new ScalaMapSerializer(mutable.HashMap() ++ _)) - // TODO: add support for immutable maps too; this is more annoying because there are many - // subclasses of immutable.Map for small maps (with <= 4 entries) - val map1 = Map[Any, Any](1 -> 1) - val map2 = Map[Any, Any](1 -> 1, 2 -> 2) - val map3 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3) - val map4 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4) - val map5 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4, 5 -> 5) - kryo.register(map1.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map2.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map3.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map4.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map5.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - // Allow the user to register their own classes by setting spark.kryo.registrator - val regCls = System.getProperty("spark.kryo.registrator") - if (regCls != null) { - logInfo("Running user registrator: " + regCls) - val classLoader = Thread.currentThread.getContextClassLoader - val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] - reg.registerClasses(kryo) + try { + Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => + logDebug("Running user registrator: " + regCls) + val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] + reg.registerClasses(kryo) + } + } catch { + case _: Exception => println("Failed to register spark.kryo.registrator") } + kryo.setClassLoader(classLoader) + // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) @@ -235,7 +149,6 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { } def newInstance(): SerializerInstance = { - this.kryo.get().setClassLoader(Thread.currentThread().getContextClassLoader) new KryoSerializerInstance(this) } -} +} \ No newline at end of file diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index c3323dcbb3..cb59bb9b65 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -1,30 +1,10 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - package spark import scala.collection.mutable -import scala.collection.immutable import org.scalatest.FunSuite import com.esotericsoftware.kryo._ -import SparkContext._ - class KryoSerializerSuite extends FunSuite { test("basic types") { val ser = (new KryoSerializer).newInstance() @@ -53,6 +33,7 @@ class KryoSerializerSuite extends FunSuite { check(Array(true, false, true)) check(Array('a', 'b', 'c')) check(Array[Int]()) + check(Array(Array("1", "2"), Array("1", "2", "3", "4"))) } test("pairs") { @@ -99,11 +80,10 @@ class KryoSerializerSuite extends FunSuite { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) - check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) } test("custom registrator") { - import spark.test._ + import KryoTest._ System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) val ser = (new KryoSerializer).newInstance() @@ -123,14 +103,14 @@ class KryoSerializerSuite extends FunSuite { val hashMap = new java.util.HashMap[String, String] hashMap.put("foo", "bar") check(hashMap) - + System.clearProperty("spark.kryo.registrator") } } -package test { +object KryoTest { case class CaseClass(i: Int, s: String) {} - + class ClassWithNoArgConstructor { var x: Int = 0 override def equals(other: Any) = other match { @@ -154,4 +134,4 @@ package test { k.register(classOf[java.util.HashMap[_, _]]) } } -} +} \ No newline at end of file diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d4d70afdd5..9920e00a67 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -170,7 +170,6 @@ object SparkBuild extends Build { "com.ning" % "compress-lzf" % "0.8.4", "org.ow2.asm" % "asm" % "4.0", "com.google.protobuf" % "protobuf-java" % "2.4.1", - "de.javakaffee" % "kryo-serializers" % "0.22", "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty), @@ -181,7 +180,9 @@ object SparkBuild extends Build { "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", - "com.codahale.metrics" % "metrics-jvm" % "3.0.0" + "com.codahale.metrics" % "metrics-jvm" % "3.0.0", + "com.twitter" % "chill_2.9.3" % "0.3.0", + "com.twitter" % "chill-java" % "0.3.0" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { From a1c515fb02646c857c607949120c7d3cb29f4dce Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:50:32 -0700 Subject: [PATCH 291/419] add copyright back in --- .../test/scala/spark/KryoSerializerSuite.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index cb59bb9b65..f6e3eec546 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark import scala.collection.mutable From fc4b025314232fbeeaacb84ec3e2491e9c4ddd1b Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:53:15 -0700 Subject: [PATCH 292/419] add test --- core/src/test/scala/spark/KryoSerializerSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index f6e3eec546..30d2d5282b 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -97,6 +97,7 @@ class KryoSerializerSuite extends FunSuite { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) + check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) } test("custom registrator") { From 30a369a808cdc16effe9c5755c74fc5a3f3d7cac Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:55:48 -0700 Subject: [PATCH 293/419] update pom.xml --- pom.xml | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 4cf9589b07..469dc64329 100644 --- a/pom.xml +++ b/pom.xml @@ -193,9 +193,14 @@ 2.4.1 - de.javakaffee - kryo-serializers - 0.22 + com.twitter + chill + 0.3.0 + + + com.twitter + chill-java + 0.3.0 com.typesafe.akka From e56aa75de0f3c00e9942f0863c0fb8c57aab5321 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 22:08:09 -0700 Subject: [PATCH 294/419] fix wrapping --- core/src/main/scala/spark/KryoSerializer.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index 2ea90d41c3..eeb2993d8a 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -27,7 +27,8 @@ import serializer.{SerializerInstance, DeserializationStream, SerializationStrea import spark.broadcast._ import spark.storage._ -private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { +private[spark] +class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) def writeObject[T](t: T): SerializationStream = { @@ -39,7 +40,8 @@ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream def close() { output.close() } } -private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { +private[spark] +class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { val input = new KryoInput(inStream) def readObject[T](): T = { From e2421c13117c7d9cc950da8c9de4903d0ea49c43 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Jul 2013 00:05:43 -0700 Subject: [PATCH 295/419] Update Chill reference in pom.xml too --- core/pom.xml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 237d988712..8c740e3887 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -57,8 +57,14 @@ protobuf-java - de.javakaffee - kryo-serializers + com.twitter + chill + 0.3.0 + + + com.twitter + chill-java + 0.3.0 com.typesafe.akka From 8eb8b52997da56acb3500fa48aac1ab9a3dfdddf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Jul 2013 08:58:02 -0700 Subject: [PATCH 296/419] Fix Chill version in Maven --- core/pom.xml | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 8c740e3887..f0c936c86a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -58,7 +58,7 @@ com.twitter - chill + chill_2.9.3 0.3.0 diff --git a/pom.xml b/pom.xml index 469dc64329..44729bd422 100644 --- a/pom.xml +++ b/pom.xml @@ -194,7 +194,7 @@ com.twitter - chill + chill_2.9.3 0.3.0 From a6de90c927e5569e30a7bc6188253bd14c32c3a9 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Thu, 25 Jul 2013 01:01:50 -0700 Subject: [PATCH 297/419] For standalone mode, get JAVA_HOME, SPARK_JAVA_OPTS, SPARK_LIBRARY_PATH from application env, not worker env --- .../main/scala/spark/deploy/worker/ExecutorRunner.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 8b51ff1c3a..47d3390928 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -51,6 +51,9 @@ private[spark] class ExecutorRunner( var process: Process = null var shutdownHook: Thread = null + private def getAppEnv(key: String): Option[String] = + appDesc.command.environment.get(key).orElse(Option(getenv(key))) + def start() { workerThread = new Thread("ExecutorRunner for " + fullId) { override def run() { fetchAndRunExecutor() } @@ -95,7 +98,7 @@ private[spark] class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = appDesc.command - val runner = Option(getenv("JAVA_HOME")).map(_ + "/bin/java").getOrElse("java") + val runner = getAppEnv("JAVA_HOME").map(_ + "/bin/java").getOrElse("java") // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++ @@ -107,10 +110,10 @@ private[spark] class ExecutorRunner( * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val libraryOpts = Option(getenv("SPARK_LIBRARY_PATH")) + val libraryOpts = getAppEnv("SPARK_LIBRARY_PATH") .map(p => List("-Djava.library.path=" + p)) .getOrElse(Nil) - val userOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil) + val userOpts = getAppEnv("SPARK_JAVA_OPTS").map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") // Figure out our classpath with the external compute-classpath script From d4bbc8bd251937a778658e09791b0416fd54336e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 25 Jul 2013 15:59:52 -0700 Subject: [PATCH 298/419] Shows totals for shuffle data and CPU time in Stage, homepage overviews including active time --- .../scala/spark/ui/exec/ExecutorsUI.scala | 10 ++--- .../main/scala/spark/ui/jobs/IndexPage.scala | 28 +++++++++++++- .../scala/spark/ui/jobs/JobProgressUI.scala | 38 ++++++++++++++++--- .../main/scala/spark/ui/jobs/StagePage.scala | 26 ++++++++++++- 4 files changed, 90 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index db1c902955..e569c04195 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -113,7 +113,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, HashSet[Long]]() + val executorToTasksActive = HashMap[String, HashSet[TaskInfo]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = @@ -122,8 +122,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId if (!executorToTasksActive.contains(eid)) - executorToTasksActive(eid) = HashSet[Long]() - executorToTasksActive(eid) += taskStart.taskInfo.taskId + executorToTasksActive(eid) = HashSet[TaskInfo]() + executorToTasksActive(eid) += taskStart.taskInfo val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -133,8 +133,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId if (!executorToTasksActive.contains(eid)) - executorToTasksActive(eid) = HashSet[Long]() - executorToTasksActive(eid) -= taskEnd.taskInfo.taskId + executorToTasksActive(eid) = HashSet[TaskInfo]() + executorToTasksActive(eid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index f31af3cda6..41186fd9bc 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -28,6 +28,7 @@ import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ import spark.storage.StorageLevel +import spark.Utils /** Page showing list of all ongoing and recently finished stages */ private[spark] class IndexPage(parent: JobProgressUI) { @@ -39,6 +40,13 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq + var activeTime = 0L + listener.stageToTasksActive.foreach {s => + s._2.foreach { t => + activeTime += t.timeRunning(System.currentTimeMillis()) + } + } + /** Special table which merges two header cells. */ def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { @@ -57,11 +65,29 @@ private[spark] class IndexPage(parent: JobProgressUI) {
        } + val summary: NodeSeq = +
        +
          +
        • + CPU time: + {parent.formatDuration(listener.totalTime + activeTime)} +
        • +
        • + Shuffle read: + {Utils.memoryBytesToString(listener.totalShuffleRead)} +
        • +
        • + Shuffle write: + {Utils.memoryBytesToString(listener.totalShuffleWrite)} +
        • +
        +
        val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) val completedStageTable = stageTable(stageRow, completedStages) val failedStageTable: NodeSeq = stageTable(stageRow, failedStages) - val content =

        Active Stages

        ++ activeStageTable ++ + val content = summary ++ +

        Active Stages

        ++ activeStageTable ++

        Completed Stages

        ++ completedStageTable ++

        Failed Stages

        ++ failedStageTable diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 6e332415db..3cc38ea403 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -65,7 +65,14 @@ private[spark] class JobProgressListener extends SparkListener { val completedStages = ListBuffer[Stage]() val failedStages = ListBuffer[Stage]() - val stageToTasksActive = HashMap[Int, HashSet[Long]]() + var totalTime = 0L + var totalShuffleRead = 0L + var totalShuffleWrite = 0L + + val stageToTime = HashMap[Int, Long]() + val stageToShuffleRead = HashMap[Int, Long]() + val stageToShuffleWrite = HashMap[Int, Long]() + val stageToTasksActive = HashMap[Int, HashSet[TaskInfo]]() val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = @@ -97,8 +104,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { val sid = taskStart.task.stageId if (!stageToTasksActive.contains(sid)) - stageToTasksActive(sid) = HashSet[Long]() - stageToTasksActive(sid) += taskStart.taskInfo.taskId + stageToTasksActive(sid) = HashSet[TaskInfo]() + stageToTasksActive(sid) += taskStart.taskInfo val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -108,8 +115,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId if (!stageToTasksActive.contains(sid)) - stageToTasksActive(sid) = HashSet[Long]() - stageToTasksActive(sid) -= taskEnd.taskInfo.taskId + stageToTasksActive(sid) = HashSet[TaskInfo]() + stageToTasksActive(sid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -119,6 +126,27 @@ private[spark] class JobProgressListener extends SparkListener { stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 (None, Some(taskEnd.taskMetrics)) } + + if (!stageToTime.contains(sid)) + stageToTime(sid) = 0L + val time = metrics.map(m => m.executorRunTime).getOrElse(1) + stageToTime(sid) += time + totalTime += time + + if (!stageToShuffleRead.contains(sid)) + stageToShuffleRead(sid) = 0L + val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => + s.remoteBytesRead).getOrElse(0L) + stageToShuffleRead(sid) += shuffleRead + totalShuffleRead += shuffleRead + + if (!stageToShuffleWrite.contains(sid)) + stageToShuffleWrite(sid) = 0L + val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => + s.shuffleBytesWritten).getOrElse(0L) + stageToShuffleWrite(sid) += shuffleWrite + totalShuffleWrite += shuffleWrite + val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList -= ((taskEnd.taskInfo, None, None)) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 654f347723..41d12b1ef4 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -52,6 +52,29 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleRead = listener.hasShuffleRead(stageId) val shuffleWrite = listener.hasShuffleWrite(stageId) + var activeTime = 0L + listener.stageToTasksActive(stageId).foreach { t => + activeTime += t.timeRunning(System.currentTimeMillis()) + } + + val summary = +
        +
          +
        • + CPU time: + {parent.formatDuration(listener.stageToTime(stageId) + activeTime)} +
        • +
        • + Shuffle read: + {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))} +
        • +
        • + Shuffle write: + {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} +
        • +
        +
        + val taskHeaders: Seq[String] = Seq("Task ID", "Status", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ @@ -98,7 +121,8 @@ private[spark] class StagePage(parent: JobProgressUI) { } val content = -

        Summary Metrics

        ++ summaryTable.getOrElse(Nil) ++

        Tasks

        ++ taskTable; + summary ++

        Summary Metrics

        ++ summaryTable.getOrElse(Nil) ++ +

        Tasks

        ++ taskTable; headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } From 22faeab261302d1506321993ecacf9f9ab9aa5f6 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 25 Jul 2013 17:14:18 -0700 Subject: [PATCH 299/419] Split Shuffle Activity overview column for read/write --- .../main/scala/spark/ui/jobs/IndexPage.scala | 25 ++++++++++++------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 41186fd9bc..9bfb9b9041 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -56,7 +56,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { Submitted Duration Tasks: Complete/Total - Shuffle Activity + Shuffle Read + Shuffle Write Stored RDD @@ -120,13 +121,18 @@ private[spark] class IndexPage(parent: JobProgressUI) { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" } - val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) - val shuffleInfo = (read, write) match { - case (true, true) => "Read/Write" - case (true, false) => "Read" - case (false, true) => "Write" - case _ => "" - } + + val shuffleRead = + if (!listener.hasShuffleRead(s.id)) + "" + else + Utils.memoryBytesToString(listener.stageToShuffleRead(s.id)) + val shuffleWrite = + if (!listener.hasShuffleWrite(s.id)) + "" + else + Utils.memoryBytesToString(listener.stageToShuffleWrite(s.id)) + val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions @@ -143,7 +149,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { case _ => }} - {shuffleInfo} + {shuffleRead} + {shuffleWrite} {if (s.rdd.getStorageLevel != StorageLevel.NONE) { {Option(s.rdd.name).getOrElse(s.rdd.id)} From 3fbe9eaac08c13527e64e4fe1ce2748838707562 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 26 Jul 2013 11:51:38 -0700 Subject: [PATCH 300/419] Displys shuffle read/write only if exists, wraps if statements, trims old vals, grabs current time once --- .../scala/spark/ui/exec/ExecutorsUI.scala | 6 ++- .../main/scala/spark/ui/jobs/IndexPage.scala | 27 ++++++------ .../scala/spark/ui/jobs/JobProgressUI.scala | 41 ++++++++++++++----- .../main/scala/spark/ui/jobs/StagePage.scala | 20 +++++---- 4 files changed, 62 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index e569c04195..b15da81f7c 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -121,8 +121,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) + if (!executorToTasksActive.contains(eid)) { executorToTasksActive(eid) = HashSet[TaskInfo]() + } executorToTasksActive(eid) += taskStart.taskInfo val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) @@ -132,8 +133,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) + if (!executorToTasksActive.contains(eid)) { executorToTasksActive(eid) = HashSet[TaskInfo]() + } executorToTasksActive(eid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 9bfb9b9041..b862c3539c 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -39,12 +39,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq + val now = System.currentTimeMillis() var activeTime = 0L - listener.stageToTasksActive.foreach {s => - s._2.foreach { t => - activeTime += t.timeRunning(System.currentTimeMillis()) - } + for (tasks <- listener.stageToTasksActive.values; t <- tasks) { + activeTime += t.timeRunning(now) } /** Special table which merges two header cells. */ @@ -73,14 +72,18 @@ private[spark] class IndexPage(parent: JobProgressUI) { CPU time: {parent.formatDuration(listener.totalTime + activeTime)} -
      • - Shuffle read: - {Utils.memoryBytesToString(listener.totalShuffleRead)} -
      • -
      • - Shuffle write: - {Utils.memoryBytesToString(listener.totalShuffleWrite)} -
      • + {if (listener.totalShuffleRead > 0) +
      • + Shuffle read: + {Utils.memoryBytesToString(listener.totalShuffleRead)} +
      • + } + {if (listener.totalShuffleWrite > 0) +
      • + Shuffle write: + {Utils.memoryBytesToString(listener.totalShuffleWrite)} +
      • + }
      val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 3cc38ea403..2fb4575551 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -93,6 +93,12 @@ private[spark] class JobProgressListener extends SparkListener { val toRemove = RETAINED_STAGES / 10 stages.takeRight(toRemove).foreach( s => { stageToTaskInfos.remove(s.id) + stageToTime.remove(s.id) + stageToShuffleRead.remove(s.id) + stageToShuffleWrite.remove(s.id) + stageToTasksActive.remove(s.id) + stageToTasksComplete.remove(s.id) + stageToTasksFailed.remove(s.id) }) stages.trimEnd(toRemove) } @@ -103,8 +109,9 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { val sid = taskStart.task.stageId - if (!stageToTasksActive.contains(sid)) + if (!stageToTasksActive.contains(sid)) { stageToTasksActive(sid) = HashSet[TaskInfo]() + } stageToTasksActive(sid) += taskStart.taskInfo val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) @@ -114,8 +121,9 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - if (!stageToTasksActive.contains(sid)) + if (!stageToTasksActive.contains(sid)) { stageToTasksActive(sid) = HashSet[TaskInfo]() + } stageToTasksActive(sid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { @@ -127,23 +135,36 @@ private[spark] class JobProgressListener extends SparkListener { (None, Some(taskEnd.taskMetrics)) } - if (!stageToTime.contains(sid)) + if (!stageToTime.contains(sid)) { stageToTime(sid) = 0L - val time = metrics.map(m => m.executorRunTime).getOrElse(1) + } + val time = if (metrics.isDefined) metrics.map(m => m.executorRunTime).getOrElse(0) else 0 stageToTime(sid) += time totalTime += time - if (!stageToShuffleRead.contains(sid)) + if (!stageToShuffleRead.contains(sid)) { stageToShuffleRead(sid) = 0L - val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => - s.remoteBytesRead).getOrElse(0L) + } + val shuffleRead = + if (metrics.isDefined) { + metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead).getOrElse(0L) + } + else { + 0L + } stageToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead - if (!stageToShuffleWrite.contains(sid)) + if (!stageToShuffleWrite.contains(sid)) { stageToShuffleWrite(sid) = 0L - val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => - s.shuffleBytesWritten).getOrElse(0L) + } + val shuffleWrite = + if (metrics.isDefined) { + metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleBytesWritten).getOrElse(0L) + } + else { + 0L + } stageToShuffleWrite(sid) += shuffleWrite totalShuffleWrite += shuffleWrite diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 41d12b1ef4..266f180a50 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -64,14 +64,18 @@ private[spark] class StagePage(parent: JobProgressUI) { CPU time: {parent.formatDuration(listener.stageToTime(stageId) + activeTime)} -
    • - Shuffle read: - {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))} -
    • -
    • - Shuffle write: - {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} -
    • + {if (listener.hasShuffleRead(stageId)) +
    • + Shuffle read: + {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))} +
    • + } + {if (listener.hasShuffleWrite(stageId)) +
    • + Shuffle write: + {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} +
    • + }
    From 743fc4e7aa8a2ca4edbe731bbefb2127d5d1a7d4 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 14:35:17 -0700 Subject: [PATCH 301/419] Fix Bug in Partition Pruning, index of Pruned Partitions should inherit from parent --- .../scala/spark/rdd/PartitionPruningRDD.scala | 6 ++-- .../spark/PartitionPruningRDDSuite.scala | 28 +++++++++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 core/src/test/scala/spark/PartitionPruningRDDSuite.scala diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 41ff62dd22..6fe004a009 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -16,8 +16,9 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo extends NarrowDependency[T](rdd) { @transient - val partitions: Array[Partition] = rdd.partitions.filter(s => partitionFilterFunc(s.index)) - .zipWithIndex.map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } + val partitions: Array[Partition] = rdd.partitions. + zipWithIndex.filter(s => partitionFilterFunc(s._2)). + map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } @@ -39,6 +40,7 @@ class PartitionPruningRDD[T: ClassManifest]( override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions + } diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala new file mode 100644 index 0000000000..a0e6413160 --- /dev/null +++ b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala @@ -0,0 +1,28 @@ +package spark + +import org.scalatest.FunSuite +import spark.SparkContext._ +import spark.rdd.PartitionPruningRDD + + +class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { + + test("Pruned Partitions inherit locality prefs correctly") { + class TestPartition(i: Int) extends Partition { + def index = i + } + val rdd = new RDD[Int](sc, Nil) { + override protected def getPartitions = { + Array[Partition]( + new TestPartition(1), + new TestPartition(2), + new TestPartition(3)) + } + def compute(split: Partition, context: TaskContext) = {Iterator()} + } + val prunedRDD = PartitionPruningRDD.create(rdd, {x => if (x==2) true else false}) + println(prunedRDD.partitions.length) + val p = prunedRDD.partitions(0) + assert(p.index == 2) + } +} \ No newline at end of file From 822aac8f5afd1342bb1cbadd7a2c2821bda81988 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 15:10:32 -0700 Subject: [PATCH 302/419] Indentation --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 1 - core/src/test/scala/spark/PartitionPruningRDDSuite.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 6fe004a009..04e22574af 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -40,7 +40,6 @@ class PartitionPruningRDD[T: ClassManifest]( override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions - } diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala index a0e6413160..4fd1f55a45 100644 --- a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala @@ -25,4 +25,4 @@ class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { val p = prunedRDD.partitions(0) assert(p.index == 2) } -} \ No newline at end of file +} From 72cf7ec0e5736d8cc5a691b36c78a79c9a4bf118 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 15:16:41 -0700 Subject: [PATCH 303/419] Indentation --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 04e22574af..eaac134ef4 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -16,9 +16,9 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo extends NarrowDependency[T](rdd) { @transient - val partitions: Array[Partition] = rdd.partitions. - zipWithIndex.filter(s => partitionFilterFunc(s._2)). - map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } + val partitions: Array[Partition] = rdd.partitions.zipWithIndex + .filter(s => partitionFilterFunc(s._2)) + .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } From 392d7474fde6f18bcd9cece5c4ed057ac1d24485 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 15:23:15 -0700 Subject: [PATCH 304/419] Code review --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 4 ++-- core/src/test/scala/spark/PartitionPruningRDDSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index eaac134ef4..fa2f5c2611 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -17,8 +17,8 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo @transient val partitions: Array[Partition] = rdd.partitions.zipWithIndex - .filter(s => partitionFilterFunc(s._2)) - .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } + .filter(s => partitionFilterFunc(s._2)) + .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala index 4fd1f55a45..88352b639f 100644 --- a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala @@ -21,8 +21,8 @@ class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { def compute(split: Partition, context: TaskContext) = {Iterator()} } val prunedRDD = PartitionPruningRDD.create(rdd, {x => if (x==2) true else false}) - println(prunedRDD.partitions.length) val p = prunedRDD.partitions(0) assert(p.index == 2) + assert(prunedRDD.partitions.length == 1) } } From 3fc64089032707393cc016b17ccd3dade20f79fe Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 26 Jul 2013 16:10:20 -0700 Subject: [PATCH 305/419] Added missing scalatest dependency --- tools/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tools/pom.xml b/tools/pom.xml index 1125aba4f1..878eb82f18 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -30,6 +30,14 @@ Spark Project Tools http://spark-project.org/ + + + org.scalatest + scalatest_${scala.version} + test + + + target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes From bd4cc52e309667296ef60557e17e8a7263c51fd8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 26 Jul 2013 17:23:18 -0700 Subject: [PATCH 306/419] Made metrics Option instead of Some, fixed NullPointerException --- .../scala/spark/ui/exec/ExecutorsUI.scala | 2 +- .../scala/spark/ui/jobs/JobProgressUI.scala | 22 +++++-------------- 2 files changed, 7 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index b15da81f7c..80d00c6873 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -144,7 +144,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { (Some(e), e.metrics) case _ => executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - (None, Some(taskEnd.taskMetrics)) + (None, Option(taskEnd.taskMetrics)) } val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 2fb4575551..e7fbff7f73 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -132,39 +132,29 @@ private[spark] class JobProgressListener extends SparkListener { (Some(e), e.metrics) case _ => stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - (None, Some(taskEnd.taskMetrics)) + (None, Option(taskEnd.taskMetrics)) } if (!stageToTime.contains(sid)) { stageToTime(sid) = 0L } - val time = if (metrics.isDefined) metrics.map(m => m.executorRunTime).getOrElse(0) else 0 + val time = metrics.map(m => m.executorRunTime).getOrElse(0) stageToTime(sid) += time totalTime += time if (!stageToShuffleRead.contains(sid)) { stageToShuffleRead(sid) = 0L } - val shuffleRead = - if (metrics.isDefined) { - metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead).getOrElse(0L) - } - else { - 0L - } + val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => + s.remoteBytesRead).getOrElse(0L) stageToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead if (!stageToShuffleWrite.contains(sid)) { stageToShuffleWrite(sid) = 0L } - val shuffleWrite = - if (metrics.isDefined) { - metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleBytesWritten).getOrElse(0L) - } - else { - 0L - } + val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => + s.shuffleBytesWritten).getOrElse(0L) stageToShuffleWrite(sid) += shuffleWrite totalShuffleWrite += shuffleWrite From f74a03c6d82f47e004d32de5bd4e17be91d35888 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 17:29:44 -0700 Subject: [PATCH 307/419] Multiple changes - Changed LogisticRegression regularization parameter to 0 - Removed println from SVM predict function - Fixed "Lasso" -> "SVM" in SVMGenerator - Added comment in Updater.scala to indicate L1 regularization leads to soft thresholding proximal function --- .../spark/mllib/classification/LogisticRegression.scala | 4 ++-- mllib/src/main/scala/spark/mllib/classification/SVM.scala | 4 +--- .../scala/spark/mllib/classification/SVMGenerator.scala | 6 +++--- mllib/src/main/scala/spark/mllib/optimization/Updater.scala | 3 +++ 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index f39c1ec52e..876197a14f 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -36,7 +36,7 @@ class LogisticRegression private (var stepSize: Double, var regParam: Double, va /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 1.0, 1.0, 100) + def this() = this(1.0, 0.0, 1.0, 100) /** * Set the step size per-iteration of SGD. Default 1.0. @@ -145,7 +145,7 @@ object LogisticRegression { numIterations: Int) : LogisticRegressionModel = { - train(input, numIterations, 1.0, 1.0, 1.0) + train(input, numIterations, 1.0, 0.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index aceb903f1d..c8da8b7c29 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -16,11 +16,9 @@ class SVMModel( val losses: Array[Double]) extends ClassificationModel { override def predict(testData: spark.RDD[Array[Double]]) = { - testData.map { x => { - println("Predicting " + x) + testData.map { x => signum(new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept) } - } } override def predict(testData: Array[Double]): Double = { diff --git a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala index a5e2837343..029f262660 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala @@ -8,11 +8,11 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.util.MLUtils -object LassoGenerator { +object SVMGenerator { def main(args: Array[String]) { if (args.length != 5) { - println("Usage: LassoGenerator " + + println("Usage: SVMGenerator " + " ") System.exit(1) } @@ -24,7 +24,7 @@ object LassoGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 3 - val sc = new SparkContext(sparkMaster, "LassoGenerator") + val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index 64c54dfb0d..26f06e86dc 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -28,6 +28,9 @@ class SimpleUpdater extends Updater { } } +/** +L1 regularization -- corresponding proximal operator is the soft-thresholding function +**/ class L1Updater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { From f0a1f95228bdb3f2f4c8a869a96c8df1a066edf3 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 17:36:14 -0700 Subject: [PATCH 308/419] Rename LogisticRegression, SVM and Lasso to *_LocalRandomSGD --- .../spark/mllib/classification/LogisticRegression.scala | 8 ++++---- mllib/src/main/scala/spark/mllib/classification/SVM.scala | 8 ++++---- mllib/src/main/scala/spark/mllib/regression/Lasso.scala | 8 ++++---- .../mllib/classification/LogisticRegressionSuite.scala | 4 ++-- .../test/scala/spark/mllib/classification/SVMSuite.scala | 4 ++-- .../test/scala/spark/mllib/regression/LassoSuite.scala | 4 ++-- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 876197a14f..243a346364 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -29,7 +29,7 @@ class LogisticRegressionModel( } } -class LogisticRegression private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -86,7 +86,7 @@ class LogisticRegression private (var stepSize: Double, var regParam: Double, va /** * Top-level methods for calling Logistic Regression. */ -object LogisticRegression { +object LogisticRegression_LocalRandomSGD { /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number @@ -107,7 +107,7 @@ object LogisticRegression { miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new LogisticRegression_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -155,7 +155,7 @@ object LogisticRegression { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegression.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LogisticRegression_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index c8da8b7c29..bf10493bf5 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -27,7 +27,7 @@ class SVMModel( } } -class SVM private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -92,7 +92,7 @@ class SVM private (var stepSize: Double, var regParam: Double, var miniBatchFrac /** * Top-level methods for calling SVM. */ -object SVM { +object SVM_LocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number @@ -113,7 +113,7 @@ object SVM { miniBatchFraction: Double) : SVMModel = { - new SVM(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -161,7 +161,7 @@ object SVM { } val sc = new SparkContext(args(0), "SVM") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = SVM.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = SVM_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index de410711a2..bb2305c811 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -26,7 +26,7 @@ class LassoModel( } } -class Lasso private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -91,7 +91,7 @@ class Lasso private (var stepSize: Double, var regParam: Double, var miniBatchFr /** * Top-level methods for calling Lasso. */ -object Lasso { +object Lasso_LocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number @@ -112,7 +112,7 @@ object Lasso { miniBatchFraction: Double) : LassoModel = { - new Lasso(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -160,7 +160,7 @@ object Lasso { } val sc = new SparkContext(args(0), "Lasso") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = Lasso.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = Lasso_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 13612e9a4a..5741906a14 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -18,7 +18,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { } // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { + test("LogisticRegression_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -45,7 +45,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(10.0) + val lr = new LogisticRegression_LocalRandomSGD().setStepSize(10.0) .setNumIterations(20) val model = lr.train(testRDD) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index e3a6681ab2..50cf260f49 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -19,7 +19,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("SVM") { + test("SVM_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -46,7 +46,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { writer_data.write("\n")}) writer_data.close() - val svm = new SVM().setStepSize(1.0) + val svm = new SVM_LocalRandomSGD().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100) diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 90fedb3e84..9836ac54c1 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -17,7 +17,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("Lasso") { + test("Lasso_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -36,7 +36,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val ls = new Lasso().setStepSize(1.0) + val ls = new Lasso_LocalRandomSGD().setStepSize(1.0) .setRegParam(0.01) .setNumIterations(20) From 10fd3949e6ccfca896ccfefe5895fda7f40944cf Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 17:49:11 -0700 Subject: [PATCH 309/419] Making ClassificationModel serializable --- .../main/scala/spark/mllib/classification/Classification.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/Classification.scala index 7f1eb21079..96d7a54f18 100644 --- a/mllib/src/main/scala/spark/mllib/classification/Classification.scala +++ b/mllib/src/main/scala/spark/mllib/classification/Classification.scala @@ -2,7 +2,7 @@ package spark.mllib.classification import spark.RDD -trait ClassificationModel { +trait ClassificationModel extends Serializable { /** * Predict values for the given data set using the model trained. * From b0bbc7f6a8da8e4c8e4e5bb656d7c7eed9b24511 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 18:57:39 -0700 Subject: [PATCH 310/419] Resolve conflicts with master, removed regParam for LogisticRegression --- .../classification/LogisticRegression.scala | 120 +++++++++++++++--- .../spark/mllib/classification/SVM.scala | 96 ++++++++++++-- .../spark/mllib/optimization/Gradient.scala | 18 ++- .../mllib/optimization/GradientDescent.scala | 37 ++++-- .../scala/spark/mllib/regression/Lasso.scala | 97 ++++++++++++-- .../LogisticRegressionSuite.scala | 108 ++++++++++++++-- 6 files changed, 412 insertions(+), 64 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 243a346364..40b96fbe3a 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.classification import spark.{Logging, RDD, SparkContext} @@ -11,32 +28,39 @@ import org.jblas.DoubleMatrix * Based on Matlab code written by John Duchi. */ class LogisticRegressionModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, - val losses: Array[Double]) extends ClassificationModel { + val stochasticLosses: Array[Double]) extends ClassificationModel { + + // Create a column vector that can be used for predictions + private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) override def predict(testData: spark.RDD[Array[Double]]) = { + // A small optimization to avoid serializing the entire model. Only the weightsMatrix + // and intercept is needed. + val localWeights = weightsMatrix + val localIntercept = intercept testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept 1.0/ (1.0 + math.exp(margin * -1)) } } override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - val margin = dataMat.mmul(this.weights).get(0) + this.intercept + val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } -class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 0.0, 1.0, 100) + def this() = this(1.0, 1.0, 100) /** * Set the step size per-iteration of SGD. Default 1.0. @@ -46,6 +70,14 @@ class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regPa this } + + + + + + + + /** * Set fraction of data to be used for each SGD iteration. Default 1.0. */ @@ -63,31 +95,77 @@ class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regPa } def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val initialWeights = Array.fill(nfeatures)(1.0) + train(input, initialWeights) + } + + def train( + input: RDD[(Double, Array[Double])], + initialWeights: Array[Double]): LogisticRegressionModel = { + // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => (y, Array(1.0, features:_*)) } - val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, regParam, miniBatchFraction) + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( + data, + new LogisticGradient(), + new SimpleUpdater(), + stepSize, + numIters, + 0.0, + initalWeightsWithIntercept, + miniBatchFraction) - val model = new LogisticRegressionModel(weightsScaled, intercept, losses) + val intercept = weights(0) + val weightsScaled = weights.tail - logInfo("Final model weights " + model.weights) + val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) + + logInfo("Final model weights " + model.weights.mkString(",")) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochastic losses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling Logistic Regression. + * NOTE(shivaram): We use multiple train methods instead of default arguments to support + * Java programs. */ object LogisticRegression_LocalRandomSGD { + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LogisticRegressionModel = + { + new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses @@ -96,18 +174,18 @@ object LogisticRegression_LocalRandomSGD { * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, stepSize: Double, - regParam: Double, + miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input) } /** @@ -117,18 +195,18 @@ object LogisticRegression_LocalRandomSGD { * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, - stepSize: Double, - regParam: Double) + stepSize: Double + ) : LogisticRegressionModel = { - train(input, numIterations, stepSize, regParam, 1.0) + train(input, numIterations, stepSize, 1.0) } /** @@ -145,7 +223,7 @@ object LogisticRegression_LocalRandomSGD { numIterations: Int) : LogisticRegressionModel = { - train(input, numIterations, 1.0, 0.0, 1.0) + train(input, numIterations, 1.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index bf10493bf5..2cd1d668eb 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.classification import scala.math.signum @@ -11,22 +28,31 @@ import org.jblas.DoubleMatrix * SVM using Stochastic Gradient Descent. */ class SVMModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, - val losses: Array[Double]) extends ClassificationModel { + val stochasticLosses: Array[Double]) extends ClassificationModel { + + // Create a column vector that can be used for predictions + private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) override def predict(testData: spark.RDD[Array[Double]]) = { + // A small optimization to avoid serializing the entire model. Only the weightsMatrix + // and intercept is needed. + val localWeights = weightsMatrix + val localIntercept = intercept testData.map { x => - signum(new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept) + signum(new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept) } } override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - signum(dataMat.dot(this.weights) + this.intercept) + signum(dataMat.dot(weightsMatrix) + this.intercept) } } + + class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -69,31 +95,77 @@ class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, va } def train(input: RDD[(Double, Array[Double])]): SVMModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val initialWeights = Array.fill(nfeatures)(1.0) + train(input, initialWeights) + } + + def train( + input: RDD[(Double, Array[Double])], + initialWeights: Array[Double]): SVMModel = { + // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => (y, Array(1.0, features:_*)) } - val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new HingeGradient(), new SquaredL2Updater(), stepSize, numIters, regParam, miniBatchFraction) + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( + data, + new HingeGradient(), + new SquaredL2Updater(), + stepSize, + numIters, + regParam, + initalWeightsWithIntercept, + miniBatchFraction) - val model = new SVMModel(weightsScaled, intercept, losses) + val intercept = weights(0) + val weightsScaled = weights.tail - logInfo("Final model weights " + model.weights) + val model = new SVMModel(weightsScaled, intercept, stochasticLosses) + + logInfo("Final model weights " + model.weights.mkString(",")) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochasticLosses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling SVM. + + */ object SVM_LocalRandomSGD { + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : SVMModel = + { + new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses @@ -151,7 +223,7 @@ object SVM_LocalRandomSGD { numIterations: Int) : SVMModel = { - train(input, numIterations, 1.0, 0.10, 1.0) + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 6ffc3b128b..4864ab7ccf 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import org.jblas.DoubleMatrix @@ -58,4 +75,3 @@ class HingeGradient extends Gradient { (DoubleMatrix.zeros(1,weights.length), 0.0) } } - diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index bd8489c386..8387d4939b 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.optimization import spark.{Logging, RDD, SparkContext} @@ -23,8 +40,9 @@ object GradientDescent { * @param miniBatchFraction - fraction of the input data set that should be used for * one iteration of SGD. Default value 1.0. * - * @return weights - Column matrix containing weights for every feature. - * @return lossHistory - Array containing the loss computed for every iteration. + * @return A tuple containing two elements. The first element is a column matrix containing + * weights for every feature, and the second element is an array containing the stochastic + * loss computed for every iteration. */ def runMiniBatchSGD( data: RDD[(Double, Array[Double])], @@ -33,16 +51,16 @@ object GradientDescent { stepSize: Double, numIters: Int, regParam: Double, - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + initialWeights: Array[Double], + miniBatchFraction: Double=1.0) : (Array[Double], Array[Double]) = { - val lossHistory = new ArrayBuffer[Double](numIters) + val stochasticLossHistory = new ArrayBuffer[Double](numIters) - val nfeatures: Int = data.take(1)(0)._2.length val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction - // Initialize weights as a column matrix - var weights = DoubleMatrix.ones(nfeatures) + // Initialize weights as a column vector + var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) var reg_val = 0.0 for (i <- 1 to numIters) { @@ -53,16 +71,17 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) + stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) weights = update._1 reg_val = update._2 - lossHistory.append(lossSum / miniBatchSize + reg_val) + stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) /*** Xinghao: The loss here is sum of lossSum computed using the weights before applying updater, and reg_val using weights after applying updater ***/ } - (weights, lossHistory.toArray) + (weights.toArray, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index bb2305c811..64364323a2 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import spark.{Logging, RDD, SparkContext} @@ -8,24 +25,34 @@ import org.jblas.DoubleMatrix /** * Lasso using Stochastic Gradient Descent. + * */ class LassoModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, - val losses: Array[Double]) extends RegressionModel { + val stochasticLosses: Array[Double]) extends RegressionModel { + + // Create a column vector that can be used for predictions + private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) override def predict(testData: spark.RDD[Array[Double]]) = { + // A small optimization to avoid serializing the entire model. Only the weightsMatrix + // and intercept is needed. + val localWeights = weightsMatrix + val localIntercept = intercept testData.map { x => - new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept + new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept } } + override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - dataMat.dot(this.weights) + this.intercept + dataMat.dot(weightsMatrix) + this.intercept } } + class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -68,31 +95,77 @@ class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, } def train(input: RDD[(Double, Array[Double])]): LassoModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val initialWeights = Array.fill(nfeatures)(1.0) + train(input, initialWeights) + } + + def train( + input: RDD[(Double, Array[Double])], + initialWeights: Array[Double]): LassoModel = { + // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => (y, Array(1.0, features:_*)) } - val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new SquaredGradient(), new L1Updater(), stepSize, numIters, regParam, miniBatchFraction) + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( + data, + new SquaredGradient(), + new L1Updater(), + stepSize, + numIters, + regParam, + initalWeightsWithIntercept, + miniBatchFraction) - val model = new LassoModel(weightsScaled, intercept, losses) + val intercept = weights(0) + val weightsScaled = weights.tail - logInfo("Final model weights " + model.weights) + val model = new LassoModel(weightsScaled, intercept, stochasticLosses) + + logInfo("Final model weights " + model.weights.mkString(",")) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochasticLosses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling Lasso. + * + * */ object Lasso_LocalRandomSGD { + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LassoModel = + { + new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses @@ -150,7 +223,7 @@ object Lasso_LocalRandomSGD { numIterations: Int) : LassoModel = { - train(input, numIterations, 1.0, 0.10, 1.0) + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 5741906a14..827ca66330 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -1,4 +1,25 @@ +<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala package spark.mllib.classification +======= +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.regression +>>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala import scala.util.Random @@ -6,7 +27,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { @@ -17,16 +37,23 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } +<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("LogisticRegression_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) +======= + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) +>>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val A = 2.0 - val B = -1.5 - // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) val unifRand = new scala.util.Random(45) val rLogis = (0 until nPoints).map { i => @@ -34,24 +61,87 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { math.log(u) - math.log(1.0-u) } - // y <- A + B*x + rlogis(100) + // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => - val yVal = A + B * x1(i) + rLogis(i) + val y: Seq[Double] = (0 until nPoints).map { i => + val yVal = offset + scale * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + val testData = (0 until nPoints).map(i => (y(i), Array(x1(i)))) + testData + } + + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() +<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala val lr = new LogisticRegression_LocalRandomSGD().setStepSize(10.0) .setNumIterations(20) +======= + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(20) +>>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala val model = lr.train(testRDD) - val weight0 = model.weights.get(0) + // Test the weights + val weight0 = model.weights(0) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) + } + + test("logistic regression with initial weights") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints, 42) + + val initialB = -1.0 + val initialWeights = Array(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(10) + + val model = lr.train(testRDD, initialWeights) + + val weight0 = model.weights(0) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } From 0c391feb73610e56beb542bbf66c56efab01bada Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Mon, 22 Jul 2013 16:39:34 -0700 Subject: [PATCH 311/419] Maximum task failures configurable --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index d72b0bfc9f..c15aeb5ff0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -67,7 +67,7 @@ private[spark] class ClusterTaskSetManager( val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toDouble // Maximum times a task is allowed to fail before failing the job - val MAX_TASK_FAILURES = 4 + val MAX_TASK_FAILURES = System.getProperty("spark.task.max.fail","4").toInt // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble From 6a47cee72104c9f6d264cb2e61ae8a38523cfdad Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Mon, 22 Jul 2013 16:41:37 -0700 Subject: [PATCH 312/419] style --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index c15aeb5ff0..685fc016b2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -67,7 +67,7 @@ private[spark] class ClusterTaskSetManager( val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toDouble // Maximum times a task is allowed to fail before failing the job - val MAX_TASK_FAILURES = System.getProperty("spark.task.max.fail","4").toInt + val MAX_TASK_FAILURES = System.getProperty("spark.task.max.fail", "4").toInt // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble From 1714693324b5a3169d2e10dc16a2598f8e14d8b1 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 12:24:41 -0700 Subject: [PATCH 313/419] Current time called once with value now --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 266f180a50..5a18559ae9 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -34,6 +34,7 @@ import spark.executor.TaskMetrics private[spark] class StagePage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt + val now = System.currentTimeMillis() def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt @@ -54,7 +55,7 @@ private[spark] class StagePage(parent: JobProgressUI) { var activeTime = 0L listener.stageToTasksActive(stageId).foreach { t => - activeTime += t.timeRunning(System.currentTimeMillis()) + activeTime += t.timeRunning(now) } val summary = From dcc4743a950c7a9b767d7157ca62f512917ee6ae Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 12:52:53 -0700 Subject: [PATCH 314/419] Moved val now to render --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 5a18559ae9..ef72f42e30 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -34,10 +34,10 @@ import spark.executor.TaskMetrics private[spark] class StagePage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt - val now = System.currentTimeMillis() def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt + val now = System.currentTimeMillis() if (!listener.stageToTaskInfos.contains(stageId)) { val content = From 5a93e3c58c69574eaac6458f8515579b5bd03fd9 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 15:55:26 -0700 Subject: [PATCH 315/419] Cleaned up code based on pwendell's suggestions --- .../scala/spark/ui/exec/ExecutorsUI.scala | 12 ++---- .../main/scala/spark/ui/jobs/IndexPage.scala | 18 ++++---- .../scala/spark/ui/jobs/JobProgressUI.scala | 43 ++++--------------- 3 files changed, 20 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 80d00c6873..948b3017db 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -121,10 +121,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) { - executorToTasksActive(eid) = HashSet[TaskInfo]() - } - executorToTasksActive(eid) += taskStart.taskInfo + val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) + activeTasks += taskStart.taskInfo val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -133,10 +131,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) { - executorToTasksActive(eid) = HashSet[TaskInfo]() - } - executorToTasksActive(eid) -= taskEnd.taskInfo + val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) + activeTasks -= taskStart.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index b862c3539c..7e504c5f9f 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -125,16 +125,14 @@ private[spark] class IndexPage(parent: JobProgressUI) { case None => "Unknown" } - val shuffleRead = - if (!listener.hasShuffleRead(s.id)) - "" - else - Utils.memoryBytesToString(listener.stageToShuffleRead(s.id)) - val shuffleWrite = - if (!listener.hasShuffleWrite(s.id)) - "" - else - Utils.memoryBytesToString(listener.stageToShuffleWrite(s.id)) + val shuffleRead = listener.stageToShuffleRead(s.id) match { + case 0 => "" + case b => Utils.memoryBytesToString(b) + } + val shuffleWrite = listener.stageToShuffleWrite(s.id) match { + case 0 => "" + case b => Utils.memoryBytesToString(b) + } val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index e7fbff7f73..09d24b6302 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -65,6 +65,7 @@ private[spark] class JobProgressListener extends SparkListener { val completedStages = ListBuffer[Stage]() val failedStages = ListBuffer[Stage]() + // Total metrics reflect metrics only for completed tasks var totalTime = 0L var totalShuffleRead = 0L var totalShuffleWrite = 0L @@ -109,10 +110,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { val sid = taskStart.task.stageId - if (!stageToTasksActive.contains(sid)) { - stageToTasksActive(sid) = HashSet[TaskInfo]() - } - stageToTasksActive(sid) += taskStart.taskInfo + val tasksActive = stageToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) + tasksActive += taskStart.taskInfo val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -121,10 +120,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - if (!stageToTasksActive.contains(sid)) { - stageToTasksActive(sid) = HashSet[TaskInfo]() - } - stageToTasksActive(sid) -= taskEnd.taskInfo + val tasksActive = stageToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) + tasksActive -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -135,24 +132,18 @@ private[spark] class JobProgressListener extends SparkListener { (None, Option(taskEnd.taskMetrics)) } - if (!stageToTime.contains(sid)) { - stageToTime(sid) = 0L - } + stageToTime.getOrElseUpdate(sid, 0L) val time = metrics.map(m => m.executorRunTime).getOrElse(0) stageToTime(sid) += time totalTime += time - if (!stageToShuffleRead.contains(sid)) { - stageToShuffleRead(sid) = 0L - } + stageToShuffleRead.getOrElseUpdate(sid, 0L) val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead).getOrElse(0L) stageToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead - if (!stageToShuffleWrite.contains(sid)) { - stageToShuffleWrite(sid) = 0L - } + stageToShuffleWrite.getOrElseUpdate(sid, 0L) val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleBytesWritten).getOrElse(0L) stageToShuffleWrite(sid) += shuffleWrite @@ -178,22 +169,4 @@ private[spark] class JobProgressListener extends SparkListener { case _ => } } - - /** Is this stage's input from a shuffle read. */ - def hasShuffleRead(stageID: Int): Boolean = { - // This is written in a slightly complicated way to avoid having to scan all tasks - for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.flatMap(m => m.shuffleReadMetrics).isDefined - } - return false // No tasks have finished for this stage - } - - /** Is this stage's output to a shuffle write. */ - def hasShuffleWrite(stageID: Int): Boolean = { - // This is written in a slightly complicated way to avoid having to scan all tasks - for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.flatMap(m => m.shuffleWriteMetrics).isDefined - } - return false // No tasks have finished for this stage - } } From c2223e68012a44e9c4bebd97dc1db772df8c597e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 15 Jul 2013 08:48:53 -0700 Subject: [PATCH 316/419] Improve catch scope and logging for client stop() This does two things: 1. Catches the more general `TimeoutException`, since those can be thrown. 2. Logs at info level when a timeout is detected. --- core/src/main/scala/spark/deploy/client/Client.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index edefa0292d..6b038f5874 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -22,7 +22,7 @@ import akka.actor._ import akka.pattern.ask import akka.util.Duration import akka.util.duration._ -import akka.pattern.AskTimeoutException +import java.util.concurrent.TimeoutException import spark.{SparkException, Logging} import akka.remote.RemoteClientLifeCycleEvent import akka.remote.RemoteClientShutdown @@ -134,7 +134,8 @@ private[spark] class Client( val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { - case e: AskTimeoutException => // Ignore it, maybe master went away + case e: TimeoutException => + logInfo("Close request to Master timed out; it may already be shut down.") } actor = null } From 8177165ac41c1f39a09fa2a0b195f03928472fe5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 27 Jul 2013 16:01:54 -0700 Subject: [PATCH 317/419] Log executor on finish --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 860a38e9f8..bbf234febd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -564,8 +564,8 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: decreaseRunningTasks(1) if (!finished(index)) { tasksFinished += 1 - logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( - tid, info.duration, tasksFinished, numTasks)) + logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format( + tid, info.duration, info.hostPort, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler try { val result = ser.deserialize[TaskResult[_]](serializedData) From bcafb36c1ebe3d8f377ba69952235ae0a829bff8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 27 Jul 2013 16:03:50 -0700 Subject: [PATCH 318/419] Slight wording change --- core/src/main/scala/spark/deploy/client/Client.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 6b038f5874..29e494f495 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -135,7 +135,7 @@ private[spark] class Client( Await.result(future, timeout) } catch { case e: TimeoutException => - logInfo("Close request to Master timed out; it may already be shut down.") + logInfo("Stop request to Master timed out; it may already be shut down.") } actor = null } From 077f2dad226b4f817cb50ad5c07702e78506a698 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 16:39:36 -0700 Subject: [PATCH 319/419] Fixed outdated bugs --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/jobs/StagePage.scala | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 948b3017db..b70153fd30 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -132,7 +132,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) - activeTasks -= taskStart.taskInfo + activeTasks -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 7e504c5f9f..c12a4fe2a5 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -125,11 +125,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { case None => "Unknown" } - val shuffleRead = listener.stageToShuffleRead(s.id) match { + val shuffleRead = listener.stageToShuffleRead.getOrElse(s.id, 0L) match { case 0 => "" case b => Utils.memoryBytesToString(b) } - val shuffleWrite = listener.stageToShuffleWrite(s.id) match { + val shuffleWrite = listener.stageToShuffleWrite.getOrElse(s.id, 0L) match { case 0 => "" case b => Utils.memoryBytesToString(b) } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index ef72f42e30..e327cb3947 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -50,8 +50,8 @@ private[spark] class StagePage(parent: JobProgressUI) { val tasks = listener.stageToTaskInfos(stageId) - val shuffleRead = listener.hasShuffleRead(stageId) - val shuffleWrite = listener.hasShuffleWrite(stageId) + val shuffleRead = listener.stageToShuffleRead(stageId) > 0 + val shuffleWrite = listener.stageToShuffleWrite(stageId) > 0 var activeTime = 0L listener.stageToTasksActive(stageId).foreach { t => @@ -65,13 +65,13 @@ private[spark] class StagePage(parent: JobProgressUI) { CPU time: {parent.formatDuration(listener.stageToTime(stageId) + activeTime)} - {if (listener.hasShuffleRead(stageId)) + {if (shuffleRead)
  • Shuffle read: {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))}
  • } - {if (listener.hasShuffleWrite(stageId)) + {if (shuffleWrite)
  • Shuffle write: {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} From f11ad72d4ee2c6821749e1bf95c46d3f2c2cd860 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 27 Jul 2013 21:11:28 -0400 Subject: [PATCH 320/419] Some fixes to Python examples (style and package name for LR) --- python/examples/als.py | 5 ++--- python/examples/kmeans.py | 3 +-- python/examples/logistic_regression.py | 5 ++--- python/examples/pi.py | 3 +-- python/examples/transitive_closure.py | 5 ++--- python/examples/wordcount.py | 3 +-- 6 files changed, 9 insertions(+), 15 deletions(-) mode change 100644 => 100755 python/examples/kmeans.py mode change 100644 => 100755 python/examples/pi.py mode change 100644 => 100755 python/examples/transitive_closure.py mode change 100644 => 100755 python/examples/wordcount.py diff --git a/python/examples/als.py b/python/examples/als.py index f2b2eee64c..a77dfb2577 100755 --- a/python/examples/als.py +++ b/python/examples/als.py @@ -48,8 +48,7 @@ def update(i, vec, mat, ratings): if __name__ == "__main__": if len(sys.argv) < 2: - print >> sys.stderr, \ - "Usage: PythonALS " + print >> sys.stderr, "Usage: als " exit(-1) sc = SparkContext(sys.argv[1], "PythonALS", pyFiles=[realpath(__file__)]) M = int(sys.argv[2]) if len(sys.argv) > 2 else 100 @@ -84,5 +83,5 @@ if __name__ == "__main__": usb = sc.broadcast(us) error = rmse(R, ms, us) - print "Iteration %d:" % i + print "Iteration %d:" % i print "\nRMSE: %5.4f\n" % error diff --git a/python/examples/kmeans.py b/python/examples/kmeans.py old mode 100644 new mode 100755 index c670556f2b..ba31af92fc --- a/python/examples/kmeans.py +++ b/python/examples/kmeans.py @@ -41,8 +41,7 @@ def closestPoint(p, centers): if __name__ == "__main__": if len(sys.argv) < 5: - print >> sys.stderr, \ - "Usage: PythonKMeans " + print >> sys.stderr, "Usage: kmeans " exit(-1) sc = SparkContext(sys.argv[1], "PythonKMeans") lines = sc.textFile(sys.argv[2]) diff --git a/python/examples/logistic_regression.py b/python/examples/logistic_regression.py index 54d227d0d3..3ac1bae4e9 100755 --- a/python/examples/logistic_regression.py +++ b/python/examples/logistic_regression.py @@ -35,7 +35,7 @@ np.random.seed(42) DataPoint = namedtuple("DataPoint", ['x', 'y']) -from lr import DataPoint # So that DataPoint is properly serialized +from logistic_regression import DataPoint # So that DataPoint is properly serialized def generateData(): @@ -48,8 +48,7 @@ def generateData(): if __name__ == "__main__": if len(sys.argv) == 1: - print >> sys.stderr, \ - "Usage: PythonLR []" + print >> sys.stderr, "Usage: logistic_regression []" exit(-1) sc = SparkContext(sys.argv[1], "PythonLR", pyFiles=[realpath(__file__)]) slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 diff --git a/python/examples/pi.py b/python/examples/pi.py old mode 100644 new mode 100755 index 33c026e824..ab0645fc2f --- a/python/examples/pi.py +++ b/python/examples/pi.py @@ -24,8 +24,7 @@ from pyspark import SparkContext if __name__ == "__main__": if len(sys.argv) == 1: - print >> sys.stderr, \ - "Usage: PythonPi []" + print >> sys.stderr, "Usage: pi []" exit(-1) sc = SparkContext(sys.argv[1], "PythonPi") slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 diff --git a/python/examples/transitive_closure.py b/python/examples/transitive_closure.py old mode 100644 new mode 100755 index 40be3b5000..744cce6651 --- a/python/examples/transitive_closure.py +++ b/python/examples/transitive_closure.py @@ -37,10 +37,9 @@ def generateGraph(): if __name__ == "__main__": if len(sys.argv) == 1: - print >> sys.stderr, \ - "Usage: PythonTC []" + print >> sys.stderr, "Usage: transitive_closure []" exit(-1) - sc = SparkContext(sys.argv[1], "PythonTC") + sc = SparkContext(sys.argv[1], "PythonTransitiveClosure") slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 tc = sc.parallelize(generateGraph(), slices).cache() diff --git a/python/examples/wordcount.py b/python/examples/wordcount.py old mode 100644 new mode 100755 index 41c846ba79..a6de22766a --- a/python/examples/wordcount.py +++ b/python/examples/wordcount.py @@ -23,8 +23,7 @@ from pyspark import SparkContext if __name__ == "__main__": if len(sys.argv) < 3: - print >> sys.stderr, \ - "Usage: PythonWordCount " + print >> sys.stderr, "Usage: wordcount " exit(-1) sc = SparkContext(sys.argv[1], "PythonWordCount") lines = sc.textFile(sys.argv[2], 1) From f5067abe85922b8075e03f39a45e4ac39267c62e Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Sat, 27 Jul 2013 23:08:00 -0700 Subject: [PATCH 321/419] changes per comments. --- .../spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- docs/configuration.md | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 685fc016b2..d541234548 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -67,7 +67,7 @@ private[spark] class ClusterTaskSetManager( val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toDouble // Maximum times a task is allowed to fail before failing the job - val MAX_TASK_FAILURES = System.getProperty("spark.task.max.fail", "4").toInt + val MAX_TASK_FAILURES = System.getProperty("spark.task.maxFailures", "4").toInt // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble diff --git a/docs/configuration.md b/docs/configuration.md index 17fdbf04d1..e217b499c7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -260,6 +260,14 @@ Apart from these, the following properties are also available, and may be useful applications). Note that any RDD that persists in memory for more than this duration will be cleared as well. + + spark.task.maxFailures + 4 + + Number of individual task failures before giving up on the job. + Should greater or equal to 1. Number of allowed retries = this value - 1. + + From 0862494d442199402e37dc968dadf9a5bec9ae78 Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Sat, 27 Jul 2013 23:16:20 -0700 Subject: [PATCH 322/419] typo --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index e217b499c7..6ca5cb99f9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -265,7 +265,7 @@ Apart from these, the following properties are also available, and may be useful 4 Number of individual task failures before giving up on the job. - Should greater or equal to 1. Number of allowed retries = this value - 1. + Should be greater than or equal to 1. Number of allowed retries = this value - 1. From ccfa362ddec1bc942785798dea41c3aac52df60f Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 10:33:57 -0700 Subject: [PATCH 323/419] Change *_LocalRandomSGD to *LocalRandomSGD --- .../classification/LogisticRegression.scala | 10 ++++---- .../spark/mllib/classification/SVM.scala | 10 ++++---- .../scala/spark/mllib/regression/Lasso.scala | 10 ++++---- .../LogisticRegressionSuite.scala | 23 +++---------------- .../spark/mllib/classification/SVMSuite.scala | 4 ++-- .../spark/mllib/regression/LassoSuite.scala | 8 +++---- 6 files changed, 24 insertions(+), 41 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 40b96fbe3a..1b093187f2 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -53,7 +53,7 @@ class LogisticRegressionModel( } } -class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var miniBatchFraction: Double, +class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -138,7 +138,7 @@ class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var miniB * NOTE(shivaram): We use multiple train methods instead of default arguments to support * Java programs. */ -object LogisticRegression_LocalRandomSGD { +object LogisticRegressionLocalRandomSGD { /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number @@ -163,7 +163,7 @@ object LogisticRegression_LocalRandomSGD { initialWeights: Array[Double]) : LogisticRegressionModel = { - new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) + new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) } /** @@ -185,7 +185,7 @@ object LogisticRegression_LocalRandomSGD { miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input) + new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input) } /** @@ -233,7 +233,7 @@ object LogisticRegression_LocalRandomSGD { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegression_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LogisticRegressionLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 2cd1d668eb..76844f6b9c 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -53,7 +53,7 @@ class SVMModel( -class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -138,7 +138,7 @@ class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, va */ -object SVM_LocalRandomSGD { +object SVMLocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number @@ -163,7 +163,7 @@ object SVM_LocalRandomSGD { initialWeights: Array[Double]) : SVMModel = { - new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) } /** @@ -185,7 +185,7 @@ object SVM_LocalRandomSGD { miniBatchFraction: Double) : SVMModel = { - new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -233,7 +233,7 @@ object SVM_LocalRandomSGD { } val sc = new SparkContext(args(0), "SVM") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = SVM_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = SVMLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index 64364323a2..1952658bb2 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -53,7 +53,7 @@ class LassoModel( } -class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class LassoLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -138,7 +138,7 @@ class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, * * */ -object Lasso_LocalRandomSGD { +object LassoLocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number @@ -163,7 +163,7 @@ object Lasso_LocalRandomSGD { initialWeights: Array[Double]) : LassoModel = { - new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) } /** @@ -185,7 +185,7 @@ object Lasso_LocalRandomSGD { miniBatchFraction: Double) : LassoModel = { - new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -233,7 +233,7 @@ object Lasso_LocalRandomSGD { } val sc = new SparkContext(args(0), "Lasso") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = Lasso_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LassoLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 827ca66330..144b8b1bc7 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -1,6 +1,3 @@ -<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala -package spark.mllib.classification -======= /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -18,8 +15,7 @@ package spark.mllib.classification * limitations under the License. */ -package spark.mllib.regression ->>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +package spark.mllib.classification import scala.util.Random @@ -37,13 +33,6 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } -<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("LogisticRegression_LocalRandomSGD") { - val nPoints = 10000 - val rnd = new Random(42) - -======= // Generate input of the form Y = logistic(offset + scale*X) def generateLogisticInput( offset: Double, @@ -51,7 +40,6 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { nPoints: Int, seed: Int): Seq[(Double, Array[Double])] = { val rnd = new Random(seed) ->>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) @@ -91,12 +79,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() -<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala - val lr = new LogisticRegression_LocalRandomSGD().setStepSize(10.0) - .setNumIterations(20) -======= - val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(20) ->>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala + val lr = new LogisticRegressionLocalRandomSGD().setStepSize(10.0).setNumIterations(20) val model = lr.train(testRDD) @@ -128,7 +111,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { testRDD.cache() // Use half as many iterations as the previous test. - val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(10) + val lr = new LogisticRegressionLocalRandomSGD().setStepSize(10.0).setNumIterations(10) val model = lr.train(testRDD, initialWeights) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 50cf260f49..0d781c310c 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -19,7 +19,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("SVM_LocalRandomSGD") { + test("SVMLocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -46,7 +46,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { writer_data.write("\n")}) writer_data.close() - val svm = new SVM_LocalRandomSGD().setStepSize(1.0) + val svm = new SVMLocalRandomSGD().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100) diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 9836ac54c1..0c39e1e09b 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -17,7 +17,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("Lasso_LocalRandomSGD") { + test("LassoLocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -36,14 +36,14 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val ls = new Lasso_LocalRandomSGD().setStepSize(1.0) + val ls = new LassoLocalRandomSGD().setStepSize(1.0) .setRegParam(0.01) .setNumIterations(20) val model = ls.train(testRDD) - val weight0 = model.weights.get(0) - val weight1 = model.weights.get(1) + val weight0 = model.weights(0) + val weight1 = model.weights(1) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") From 72ff62a37c7310bab02f0231e91d3ba4d423217a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 22:21:04 -0400 Subject: [PATCH 324/419] Two fixes to IPython support: - Don't attempt to run worker processes with ipython (that can cause some crashes as ipython prints things to standard out) - Allow passing some IPYTHON_OPTS to launch things like the notebook --- pyspark | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/pyspark b/pyspark index 37a355462e..801239c108 100755 --- a/pyspark +++ b/pyspark @@ -53,9 +53,13 @@ if [[ "$SPARK_LAUNCH_WITH_SCALA" != "0" ]] ; then export SPARK_LAUNCH_WITH_SCALA=1 fi -if [[ "$IPYTHON" = "1" ]] ; then - export PYSPARK_PYTHON="ipython" - exec "$PYSPARK_PYTHON" -i -c "%run $PYTHONSTARTUP" -else - exec "$PYSPARK_PYTHON" "$@" +if [ -n "$IPYTHON_OPTS" ]; then + IPYTHON=1 +fi + +if [[ "$IPYTHON" = "1" ]] ; then + IPYTHON_OPTS=${IPYTHON_OPTS:--i} + exec ipython "$IPYTHON_OPTS" -c "%run $PYTHONSTARTUP" +else + exec "$PYSPARK_PYTHON" "$@" fi From 29e042940ac79e42e2f8818ceda6a962a76948ac Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 20:39:52 -0700 Subject: [PATCH 325/419] Move data generators to util --- .../LassoGenerator.scala => util/LassoDataGenerator.scala} | 0 .../SVMGenerator.scala => util/SVMDataGenerator.scala} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename mllib/src/main/scala/spark/mllib/{regression/LassoGenerator.scala => util/LassoDataGenerator.scala} (100%) rename mllib/src/main/scala/spark/mllib/{classification/SVMGenerator.scala => util/SVMDataGenerator.scala} (100%) diff --git a/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala From 67de051bbb81096dc37ea6f92a82a9224b4af61e Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 21:09:56 -0700 Subject: [PATCH 326/419] SVMSuite and LassoSuite rewritten to follow closely with LogisticRegressionSuite --- .../spark/mllib/classification/SVMSuite.scala | 101 ++++++++++++++---- .../spark/mllib/regression/LassoSuite.scala | 97 ++++++++++++++--- 2 files changed, 162 insertions(+), 36 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 0d781c310c..2a23825acc 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.classification import scala.util.Random @@ -7,7 +24,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ import java.io._ @@ -19,43 +35,82 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } + // Generate noisy input of the form Y = signum(x.dot(weights) + intercept + noise) + def generateSVMInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) + val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) + val y = x.map(xi => + signum((xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + ) + y zip x + } + + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + test("SVMLocalRandomSGD") { val nPoints = 10000 - val rnd = new Random(42) - - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) val A = 2.0 val B = -1.5 val C = 1.0 - val y = (0 until nPoints).map { i => - signum(A + B * x1(i) + C * x2(i) + 0.0*rnd.nextGaussian()) - } - - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + val testData = generateSVMInput(A, Array[Double](B,C), nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val writer_data = new PrintWriter(new File("svmtest.dat")) - testData.foreach(yx => { - writer_data.write(yx._1 + "") - yx._2.foreach(xi => writer_data.write("\t" + xi)) - writer_data.write("\n")}) - writer_data.close() - - val svm = new SVMLocalRandomSGD().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100) + val svm = new SVMLocalRandomSGD().setStepSize(1.0).setRegParam(1.0).setNumIterations(100) val model = svm.train(testRDD) - val yPredict = (0 until nPoints).map(i => model.predict(Array(x1(i),x2(i)))) + val validationData = generateSVMInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) - val accuracy = ((y zip yPredict).map(yy => if (yy._1==yy._2) 1 else 0).reduceLeft(_+_).toDouble / nPoints.toDouble) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) - assert(accuracy >= 0.90, "Accuracy (" + accuracy + ") too low") + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) + } + + test("SVMLocalRandomSGD with initial weights") { + val nPoints = 10000 + + val A = 2.0 + val B = -1.5 + val C = 1.0 + + val testData = generateSVMInput(A, Array[Double](B,C), nPoints, 42) + + val initialB = -1.0 + val initialC = -1.0 + val initialWeights = Array(initialB,initialC) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val svm = new SVMLocalRandomSGD().setStepSize(1.0).setRegParam(1.0).setNumIterations(100) + + val model = svm.train(testRDD, initialWeights) + + val validationData = generateSVMInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 0c39e1e09b..33e87dfd9f 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.regression import scala.util.Random @@ -6,7 +23,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ class LassoSuite extends FunSuite with BeforeAndAfterAll { @@ -17,35 +33,90 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } + // Generate noisy input of the form Y = x.dot(weights) + intercept + noise + def generateLassoInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) + val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) + val y = x.map(xi => (xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + y zip x + } + + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + test("LassoLocalRandomSGD") { val nPoints = 10000 - val rnd = new Random(42) - - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) val A = 2.0 val B = -1.5 val C = 1.0e-2 - val y = (0 until nPoints).map { i => - A + B * x1(i) + C * x2(i) + 0.1*rnd.nextGaussian() - } - - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + val testData = generateLassoInput(A, Array[Double](B,C), nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val ls = new LassoLocalRandomSGD().setStepSize(1.0) - .setRegParam(0.01) - .setNumIterations(20) + val ls = new LassoLocalRandomSGD().setStepSize(1.0).setRegParam(0.01).setNumIterations(20) val model = ls.train(testRDD) val weight0 = model.weights(0) val weight1 = model.weights(1) + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + + val validationData = generateLassoInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) + } + + test("LassoLocalRandomSGD with initial weights") { + val nPoints = 10000 + + val A = 2.0 + val B = -1.5 + val C = 1.0e-2 + + val testData = generateLassoInput(A, Array[Double](B,C), nPoints, 42) + + val initialB = -1.0 + val initialC = -1.0 + val initialWeights = Array(initialB,initialC) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val ls = new LassoLocalRandomSGD().setStepSize(1.0).setRegParam(0.01).setNumIterations(20) + + val model = ls.train(testRDD, initialWeights) + + val weight0 = model.weights(0) + val weight1 = model.weights(1) assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + + val validationData = generateLassoInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } From 9398dced0331c0ec098ef5eb4616571874ceefb6 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 21:39:19 -0700 Subject: [PATCH 327/419] Changed Classification to return Int instead of Double Also minor changes to formatting and comments --- .../mllib/classification/Classification.scala | 8 ++++---- .../classification/LogisticRegression.scala | 16 ++++------------ .../scala/spark/mllib/classification/SVM.scala | 8 ++++---- .../spark/mllib/optimization/Gradient.scala | 4 ++-- .../mllib/optimization/GradientDescent.scala | 8 ++++---- .../scala/spark/mllib/optimization/Updater.scala | 12 ++++++++++-- .../spark/mllib/classification/SVMSuite.scala | 2 -- 7 files changed, 28 insertions(+), 30 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/Classification.scala index 96d7a54f18..d6154b66ae 100644 --- a/mllib/src/main/scala/spark/mllib/classification/Classification.scala +++ b/mllib/src/main/scala/spark/mllib/classification/Classification.scala @@ -7,15 +7,15 @@ trait ClassificationModel extends Serializable { * Predict values for the given data set using the model trained. * * @param testData RDD representing data points to be predicted - * @return RDD[Double] where each entry contains the corresponding prediction + * @return RDD[Int] where each entry contains the corresponding prediction */ - def predict(testData: RDD[Array[Double]]): RDD[Double] + def predict(testData: RDD[Array[Double]]): RDD[Int] /** * Predict values for a single data point using the model trained. * * @param testData array representing a single data point - * @return Double prediction from the trained model + * @return Int prediction from the trained model */ - def predict(testData: Array[Double]): Double + def predict(testData: Array[Double]): Int } diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 1b093187f2..0a7effb1d7 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -35,21 +35,21 @@ class LogisticRegressionModel( // Create a column vector that can be used for predictions private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) - override def predict(testData: spark.RDD[Array[Double]]) = { + override def predict(testData: spark.RDD[Array[Double]]): RDD[Int] = { // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. val localWeights = weightsMatrix val localIntercept = intercept testData.map { x => val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept - 1.0/ (1.0 + math.exp(margin * -1)) + (1.0/ (1.0 + math.exp(margin * -1))).toInt } } - override def predict(testData: Array[Double]): Double = { + override def predict(testData: Array[Double]): Int = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept - 1.0/ (1.0 + math.exp(margin * -1)) + (1.0/ (1.0 + math.exp(margin * -1))).toInt } } @@ -70,14 +70,6 @@ class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBa this } - - - - - - - - /** * Set fraction of data to be used for each SGD iteration. Default 1.0. */ diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 76844f6b9c..30766a4c64 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -35,19 +35,19 @@ class SVMModel( // Create a column vector that can be used for predictions private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) - override def predict(testData: spark.RDD[Array[Double]]) = { + override def predict(testData: spark.RDD[Array[Double]]): RDD[Int] = { // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. val localWeights = weightsMatrix val localIntercept = intercept testData.map { x => - signum(new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept) + signum(new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept).toInt } } - override def predict(testData: Array[Double]): Double = { + override def predict(testData: Array[Double]): Int = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - signum(dataMat.dot(weightsMatrix) + this.intercept) + signum(dataMat.dot(weightsMatrix) + this.intercept).toInt } } diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 4864ab7ccf..22b2ec5ed6 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -70,8 +70,8 @@ class HingeGradient extends Gradient { val dotProduct = data.dot(weights) if (1.0 > label * dotProduct) - (data.mul(-label), 1.0 - label * dotProduct) + (data.mul(-label), 1.0 - label * dotProduct) else - (DoubleMatrix.zeros(1,weights.length), 0.0) + (DoubleMatrix.zeros(1,weights.length), 0.0) } } diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 8387d4939b..d4b83a1456 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -76,10 +76,10 @@ object GradientDescent { weights = update._1 reg_val = update._2 stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) - /*** - Xinghao: The loss here is sum of lossSum computed using the weights before applying updater, - and reg_val using weights after applying updater - ***/ + /* + * NOTE(Xinghao): The loss here is sum of lossSum computed using the weights before applying updater, + * and reg_val using weights after applying updater + */ } (weights.toArray, stochasticLossHistory.toArray) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index cd344a6680..188fe7d972 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -46,17 +46,25 @@ class SimpleUpdater extends Updater { } /** -L1 regularization -- corresponding proximal operator is the soft-thresholding function +* L1 regularization -- corresponding proximal operator is the soft-thresholding function +* That is, each weight component is shrunk towards 0 by shrinkageVal +* If w > shrinkageVal, set weight component to w-shrinkageVal. +* If w < -shrinkageVal, set weight component to w+shrinkageVal. +* If -shrinkageVal < w < shrinkageVal, set weight component to 0. +* Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) **/ class L1Updater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) val normGradient = gradient.mul(thisIterStepSize) + // Take gradient step val newWeights = weightsOld.sub(normGradient) + // Soft thresholding + val shrinkageVal = regParam * thisIterStepSize (0 until newWeights.length).foreach(i => { val wi = newWeights.get(i) - newWeights.put(i, signum(wi) * max(0.0, abs(wi) - regParam * thisIterStepSize)) + newWeights.put(i, signum(wi) * max(0.0, abs(wi) - shrinkageVal)) }) (newWeights, newWeights.norm1 * regParam) } diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 2a23825acc..91c037e9b1 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -25,8 +25,6 @@ import org.scalatest.FunSuite import spark.SparkContext -import java.io._ - class SVMSuite extends FunSuite with BeforeAndAfterAll { val sc = new SparkContext("local", "test") From 96e04f4cb7de3a7c9d31aa7acba496d81066634e Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 22:12:39 -0700 Subject: [PATCH 328/419] Fixed SVM and LR train functions to take Int instead of Double for Classification --- .../classification/LogisticRegression.scala | 17 ++++++++--------- .../scala/spark/mllib/classification/SVM.scala | 16 ++++++++-------- .../LogisticRegressionSuite.scala | 10 +++++----- 3 files changed, 21 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 0a7effb1d7..cbc0d03ae1 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -86,19 +86,19 @@ class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBa this } - def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + def train(input: RDD[(Int, Array[Double])]): LogisticRegressionModel = { val nfeatures: Int = input.take(1)(0)._2.length val initialWeights = Array.fill(nfeatures)(1.0) train(input, initialWeights) } def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], initialWeights: Array[Double]): LogisticRegressionModel = { // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => - (y, Array(1.0, features:_*)) + (y.toDouble, Array(1.0, features:_*)) } val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) @@ -141,13 +141,12 @@ object LogisticRegressionLocalRandomSGD { * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. * @param initialWeights Initial set of weights to be used. Array should be equal in size to * the number of features in the data. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, @@ -170,7 +169,7 @@ object LogisticRegressionLocalRandomSGD { * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, @@ -192,7 +191,7 @@ object LogisticRegressionLocalRandomSGD { * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double ) @@ -211,7 +210,7 @@ object LogisticRegressionLocalRandomSGD { * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int) : LogisticRegressionModel = { @@ -224,7 +223,7 @@ object LogisticRegressionLocalRandomSGD { System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)).map(yx => (yx._1.toInt, yx._2)) val model = LogisticRegressionLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 30766a4c64..15b689e7e0 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -94,19 +94,19 @@ class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, var this } - def train(input: RDD[(Double, Array[Double])]): SVMModel = { + def train(input: RDD[(Int, Array[Double])]): SVMModel = { val nfeatures: Int = input.take(1)(0)._2.length val initialWeights = Array.fill(nfeatures)(1.0) train(input, initialWeights) } def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], initialWeights: Array[Double]): SVMModel = { // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => - (y, Array(1.0, features:_*)) + (y.toDouble, Array(1.0, features:_*)) } val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) @@ -155,7 +155,7 @@ object SVMLocalRandomSGD { * the number of features in the data. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, regParam: Double, @@ -178,7 +178,7 @@ object SVMLocalRandomSGD { * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, regParam: Double, @@ -200,7 +200,7 @@ object SVMLocalRandomSGD { * @return a SVMModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, regParam: Double) @@ -219,7 +219,7 @@ object SVMLocalRandomSGD { * @return a SVMModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int) : SVMModel = { @@ -232,7 +232,7 @@ object SVMLocalRandomSGD { System.exit(1) } val sc = new SparkContext(args(0), "SVM") - val data = MLUtils.loadLabeledData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)).map(yx => (yx._1.toInt, yx._2)) val model = SVMLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 144b8b1bc7..3aa9fe6d12 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -38,7 +38,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { offset: Double, scale: Double, nPoints: Int, - seed: Int): Seq[(Double, Array[Double])] = { + seed: Int): Seq[(Int, Array[Double])] = { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) @@ -51,19 +51,19 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) - val y: Seq[Double] = (0 until nPoints).map { i => + val y: Seq[Int] = (0 until nPoints).map { i => val yVal = offset + scale * x1(i) + rLogis(i) - if (yVal > 0) 1.0 else 0.0 + if (yVal > 0) 1 else 0 } val testData = (0 until nPoints).map(i => (y(i), Array(x1(i)))) testData } - def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => // A prediction is off if the prediction is more than 0.5 away from expected value. - math.abs(prediction - expected) > 0.5 + math.abs(prediction.toDouble - expected.toDouble) > 0.5 }.size // At least 80% of the predictions should be on. assert(numOffPredictions < input.length / 5) From c823ee1e2bea7cde61cb4411a0f0db91f1df2af2 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 22:17:53 -0700 Subject: [PATCH 329/419] Replace map-reduce with dot operator using DoubleMatrix --- .../main/scala/spark/mllib/util/LassoDataGenerator.scala | 4 ++-- .../main/scala/spark/mllib/util/SVMDataGenerator.scala | 6 ++++-- .../test/scala/spark/mllib/classification/SVMSuite.scala | 9 ++++++--- .../test/scala/spark/mllib/regression/LassoSuite.scala | 7 ++++++- 4 files changed, 18 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala index d2d3bb33c7..1ff32d2e7f 100644 --- a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala @@ -26,7 +26,7 @@ object LassoGenerator { val sc = new SparkContext(sparkMaster, "LassoGenerator") val globalRnd = new Random(94720) - val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -34,7 +34,7 @@ object LassoGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val y = ((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1 + val y = (new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1 (y, x) } diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala index 029f262660..ced52093f5 100644 --- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala @@ -8,6 +8,8 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.util.MLUtils +import org.jblas.DoubleMatrix + object SVMGenerator { def main(args: Array[String]) { @@ -27,7 +29,7 @@ object SVMGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -35,7 +37,7 @@ object SVMGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val y = signum(((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1) + val y = signum((new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1) (y, x) } diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 91c037e9b1..3f00398a0a 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -25,6 +25,8 @@ import org.scalatest.FunSuite import spark.SparkContext +import org.jblas.DoubleMatrix + class SVMSuite extends FunSuite with BeforeAndAfterAll { val sc = new SparkContext("local", "test") @@ -38,16 +40,17 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { intercept: Double, weights: Array[Double], nPoints: Int, - seed: Int): Seq[(Double, Array[Double])] = { + seed: Int): Seq[(Int, Array[Double])] = { val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) val y = x.map(xi => - signum((xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + signum((new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian()).toInt ) y zip x } - def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => // A prediction is off if the prediction is more than 0.5 away from expected value. math.abs(prediction - expected) > 0.5 diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 33e87dfd9f..cf2b067d40 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -24,6 +24,8 @@ import org.scalatest.FunSuite import spark.SparkContext +import org.jblas.DoubleMatrix + class LassoSuite extends FunSuite with BeforeAndAfterAll { val sc = new SparkContext("local", "test") @@ -40,8 +42,11 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { nPoints: Int, seed: Int): Seq[(Double, Array[Double])] = { val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) - val y = x.map(xi => (xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + val y = x.map(xi => + (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian() + ) y zip x } From b9d6783f36d527f5082bf13a4ee6fd108e97795c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:28:42 -0400 Subject: [PATCH 330/419] Optimize Python take() to not compute entire first partition --- .../scala/spark/api/python/PythonRDD.scala | 64 +++++++++++-------- python/pyspark/rdd.py | 15 +++-- 2 files changed, 45 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index af10822dbd..2dd79f7100 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -63,34 +63,42 @@ private[spark] class PythonRDD[T: ClassManifest]( // Start a thread to feed the process input from our parent's iterator new Thread("stdin writer for " + pythonExec) { override def run() { - SparkEnv.set(env) - val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) - val dataOut = new DataOutputStream(stream) - val printOut = new PrintWriter(stream) - // Partition index - dataOut.writeInt(split.index) - // sparkFilesDir - PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) - // Broadcast variables - dataOut.writeInt(broadcastVars.length) - for (broadcast <- broadcastVars) { - dataOut.writeLong(broadcast.id) - dataOut.writeInt(broadcast.value.length) - dataOut.write(broadcast.value) + try { + SparkEnv.set(env) + val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) + val dataOut = new DataOutputStream(stream) + val printOut = new PrintWriter(stream) + // Partition index + dataOut.writeInt(split.index) + // sparkFilesDir + PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) + // Broadcast variables + dataOut.writeInt(broadcastVars.length) + for (broadcast <- broadcastVars) { + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) + } + dataOut.flush() + // Serialized user code + for (elem <- command) { + printOut.println(elem) + } + printOut.flush() + // Data values + for (elem <- parent.iterator(split, context)) { + PythonRDD.writeAsPickle(elem, dataOut) + } + dataOut.flush() + printOut.flush() + worker.shutdownOutput() + } catch { + case e: IOException => + // This can happen for legitimate reasons if the Python code stops returning data before we are done + // passing elements through, e.g., for take(). Just log a message to say it happened. + logInfo("stdin writer to Python finished early") + logDebug("stdin writer to Python finished early", e) } - dataOut.flush() - // Serialized user code - for (elem <- command) { - printOut.println(elem) - } - printOut.flush() - // Data values - for (elem <- parent.iterator(split, context)) { - PythonRDD.writeAsPickle(elem, dataOut) - } - dataOut.flush() - printOut.flush() - worker.shutdownOutput() } }.start() @@ -297,7 +305,7 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) Utils.checkHost(serverHost, "Expected hostname") val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - + override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList override def addInPlace(val1: JList[Array[Byte]], val2: JList[Array[Byte]]) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c6a6b24c5a..6efa61aa66 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -386,13 +386,16 @@ class RDD(object): >>> sc.parallelize([2, 3, 4, 5, 6]).take(10) [2, 3, 4, 5, 6] """ + def takeUpToNum(iterator): + taken = 0 + while taken < num: + yield next(iterator) + taken += 1 + # Take only up to num elements from each partition we try + mapped = self.mapPartitions(takeUpToNum) items = [] - for partition in range(self._jrdd.splits().size()): - iterator = self.ctx._takePartition(self._jrdd.rdd(), partition) - # Each item in the iterator is a string, Python object, batch of - # Python objects. Regardless, it is sufficient to take `num` - # of these objects in order to collect `num` Python objects: - iterator = iterator.take(num) + for partition in range(mapped._jrdd.splits().size()): + iterator = self.ctx._takePartition(mapped._jrdd.rdd(), partition) items.extend(self._collect_iterator_through_file(iterator)) if len(items) >= num: break From b5ec35562210c8e7ca4fea07a0d46cb255988c0d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:38:56 -0400 Subject: [PATCH 331/419] Optimize Python foreach() to not return as many objects --- python/pyspark/rdd.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6efa61aa66..4aafe35d13 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -267,7 +267,11 @@ class RDD(object): >>> def f(x): print x >>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f) """ - self.map(f).collect() # Force evaluation + def processPartition(iterator): + for x in iterator: + f(x) + yield None + self.mapPartitions(processPartition).collect() # Force evaluation def collect(self): """ From 96b50e82dc0db501ab94748a23ae3237f1d82034 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:39:27 -0400 Subject: [PATCH 332/419] Allow python/run-tests to run from any directory --- python/run-tests | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/run-tests b/python/run-tests index 1ee947d414..6643faa2e0 100755 --- a/python/run-tests +++ b/python/run-tests @@ -21,6 +21,9 @@ # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; cd ../; pwd)" +# CD into the python directory to find things on the right path +cd "$FWDIR/python" + FAILED=0 $FWDIR/pyspark pyspark/rdd.py From d75c3086951f603ec30b2527c24559e053ed7f25 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:50:38 -0400 Subject: [PATCH 333/419] Use None instead of empty string as it's slightly smaller/faster --- python/pyspark/rdd.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 4aafe35d13..8734cacb0b 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -160,7 +160,7 @@ class RDD(object): >>> sorted(sc.parallelize([1, 1, 2, 3]).distinct().collect()) [1, 2, 3] """ - return self.map(lambda x: (x, "")) \ + return self.map(lambda x: (x, None)) \ .reduceByKey(lambda x, _: x) \ .map(lambda (x, _): x) From feba7ee540fca28872957120e5e39b9e36466953 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 00:09:11 -0400 Subject: [PATCH 334/419] SPARK-815. Python parallelize() should split lists before batching One unfortunate consequence of this fix is that we materialize any collections that are given to us as generators, but this seems necessary to get reasonable behavior on small collections. We could add a batchSize parameter later to bypass auto-computation of batch size if this becomes a problem (e.g. if users really want to parallelize big generators nicely) --- python/pyspark/context.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 2f741cb345..c2b49ff37a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -141,14 +141,21 @@ class SparkContext(object): def parallelize(self, c, numSlices=None): """ Distribute a local Python collection to form an RDD. + + >>> sc.parallelize(range(5), 5).glom().collect() + [[0], [1], [2], [3], [4]] """ numSlices = numSlices or self.defaultParallelism # Calling the Java parallelize() method with an ArrayList is too slow, # because it sends O(n) Py4J commands. As an alternative, serialized # objects are written to a file and loaded through textFile(). tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) - if self.batchSize != 1: - c = batched(c, self.batchSize) + # Make sure we distribute data evenly if it's smaller than self.batchSize + if "__len__" not in dir(c): + c = list(c) # Make it a list so we can compute its length + batchSize = min(len(c) // numSlices, self.batchSize) + if batchSize > 1: + c = batched(c, batchSize) for x in c: write_with_length(dump_pickle(x), tempFile) tempFile.close() From 497f55755f8374dd553f21cc3c5edcc6e3f5acce Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 02:50:48 -0400 Subject: [PATCH 335/419] Add docs about ipython --- docs/python-programming-guide.md | 34 +++++++++++++++++++++++++++++--- 1 file changed, 31 insertions(+), 3 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index e8aaac74d0..794bff5647 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -10,6 +10,7 @@ To learn the basics of Spark, we recommend reading through the easy to follow even if you don't know Scala. This guide will show how to use the Spark features described there in Python. + # Key Differences in the Python API There are a few key differences between the Python and Scala APIs: @@ -50,6 +51,7 @@ PySpark will automatically ship these functions to workers, along with any objec Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers. The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers. + # Installing and Configuring PySpark PySpark requires Python 2.6 or higher. @@ -81,16 +83,41 @@ The Python shell can be used explore data interactively and is a simple way to l >>> help(pyspark) # Show all pyspark functions {% endhighlight %} -By default, the `pyspark` shell creates SparkContext that runs jobs locally. -To connect to a non-local cluster, set the `MASTER` environment variable. +By default, the `pyspark` shell creates SparkContext that runs jobs locally on a single core. +To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable. For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html): {% highlight bash %} $ MASTER=spark://IP:PORT ./pyspark {% endhighlight %} +Or, to use four cores on the local machine: -# Standalone Use +{% highlight bash %} +$ MASTER=local[4] ./pyspark +{% endhighlight %} + + +## IPython + +It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter. +To do this, simply set the `IPYTHON` variable to `1` when running `pyspark`: + +{% highlight bash %} +$ IPYTHON=1 ./pyspark +{% endhighlight %} + +Alternatively, you can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch +the [IPython Notebook](http://ipython.org/notebook.html) with PyLab graphing support: + +{% highlight bash %} +$ IPYTHON_OPTS="notebook --pylab inline" ./pyspark +{% endhighlight %} + +IPython also works on a cluster or on multiple cores if you set the `MASTER` environment variable. + + +# Standalone Programs PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `pyspark`. The Quick Start guide includes a [complete example](quick-start.html#a-standalone-job-in-python) of a standalone Python job. @@ -105,6 +132,7 @@ sc = SparkContext("local", "Job Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines. Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. + # Where to Go from Here PySpark includes several sample programs in the [`python/examples` folder](https://github.com/mesos/spark/tree/master/python/examples). From 75f375730025788a5982146d97bf3df9ef69ab23 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:19:56 -0700 Subject: [PATCH 336/419] Fix rounding error in LogisticRegression.scala --- .../spark/mllib/classification/LogisticRegression.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index cbc0d03ae1..bc1c327729 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -21,6 +21,8 @@ import spark.{Logging, RDD, SparkContext} import spark.mllib.optimization._ import spark.mllib.util.MLUtils +import scala.math.round + import org.jblas.DoubleMatrix /** @@ -42,14 +44,14 @@ class LogisticRegressionModel( val localIntercept = intercept testData.map { x => val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept - (1.0/ (1.0 + math.exp(margin * -1))).toInt + round(1.0/ (1.0 + math.exp(margin * -1))).toInt } } override def predict(testData: Array[Double]): Int = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept - (1.0/ (1.0 + math.exp(margin * -1))).toInt + round(1.0/ (1.0 + math.exp(margin * -1))).toInt } } From 3a8d07df8ca5bccdbed178991dd12fde74802542 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:20:26 -0700 Subject: [PATCH 337/419] Deleting extra LogisticRegressionGenerator and RidgeRegressionGenerator --- .../LogisticRegressionGenerator.scala | 41 -------------- .../regression/RidgeRegressionGenerator.scala | 55 ------------------- 2 files changed, 96 deletions(-) delete mode 100644 mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala deleted file mode 100644 index cde1148adf..0000000000 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala +++ /dev/null @@ -1,41 +0,0 @@ -package spark.mllib.classification - -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils - -object LogisticRegressionGenerator { - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: LogisticRegressionGenerator " + - " ") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 - val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 - val parts: Int = if (args.length > 4) args(4).toInt else 2 - val eps = 3 - - val sc = new SparkContext(sparkMaster, "LogisticRegressionGenerator") - - val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => - val rnd = new Random(42 + idx) - - val y = if (idx % 2 == 0) 0 else 1 - val x = Array.fill[Double](nfeatures) { - rnd.nextGaussian() + (y * eps) - } - (y, x) - } - - MLUtils.saveLabeledData(data, outputPath) - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala deleted file mode 100644 index b83f505d8e..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ /dev/null @@ -1,55 +0,0 @@ -package spark.mllib.regression - -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils - - -object RidgeRegressionGenerator { - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: RidgeRegressionGenerator " + - " ") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 - val nfeatures: Int = if (args.length > 3) args(3).toInt else 100 - val parts: Int = if (args.length > 4) args(4).toInt else 2 - val eps = 10 - - org.jblas.util.Random.seed(42) - val sc = new SparkContext(sparkMaster, "RidgeRegressionGenerator") - - // Random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - w.put(0, 0, 10) - w.put(1, 0, 10) - - val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until parts, parts).flatMap { p => - org.jblas.util.Random.seed(42 + p) - val examplesInPartition = nexamples / parts - - val X = DoubleMatrix.rand(examplesInPartition, nfeatures) - val y = X.mmul(w) - - val rnd = new Random(42 + p) - - val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) - val yObs = new DoubleMatrix(normalValues).addi(y) - - Iterator.tabulate(examplesInPartition) { i => - (yObs.get(i, 0), X.getRow(i).toArray) - } - } - - MLUtils.saveLabeledData(data, outputPath) - sc.stop() - } -} From 07f17439a52b65d4f5ef8c8d80bc25dadc0182a8 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:22:31 -0700 Subject: [PATCH 338/419] Fix validatePrediction functions for Classification models Classifiers return categorical (Int) values that should be compared directly --- .../spark/mllib/classification/LogisticRegressionSuite.scala | 3 +-- mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 3aa9fe6d12..d3fe58a382 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -62,8 +62,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => - // A prediction is off if the prediction is more than 0.5 away from expected value. - math.abs(prediction.toDouble - expected.toDouble) > 0.5 + (prediction != expected) }.size // At least 80% of the predictions should be on. assert(numOffPredictions < input.length / 5) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 3f00398a0a..d546e0729e 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -52,8 +52,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => - // A prediction is off if the prediction is more than 0.5 away from expected value. - math.abs(prediction - expected) > 0.5 + (prediction != expected) }.size // At least 80% of the predictions should be on. assert(numOffPredictions < input.length / 5) From 2b2630ba3c621f4121da8e76fe6fcfa69d3eb74c Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:22:49 -0700 Subject: [PATCH 339/419] Style fix Lines shortened to < 100 characters --- mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala | 3 ++- mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala index 1ff32d2e7f..ef4f42a494 100644 --- a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala @@ -26,7 +26,8 @@ object LassoGenerator { val sc = new SparkContext(sparkMaster, "LassoGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) + val trueWeights = new DoubleMatrix(1, nfeatures+1, + Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala index ced52093f5..00a54d9a70 100644 --- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala @@ -29,7 +29,8 @@ object SVMGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) + val trueWeights = new DoubleMatrix(1, nfeatures+1, + Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) From 43a2cc15c0539c14732dc10220fc8bb2f152a111 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 13:37:24 -0700 Subject: [PATCH 340/419] Use Bootstrap progress bars in web UI --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index f31af3cda6..16390708ca 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -76,16 +76,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { } def makeProgressBar(completed: Int, total: Int): Seq[Node] = { - val width=130 - val height=15 - val completeWidth = (completed.toDouble / total) * width + val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) - - - - +
    +
    +
    } From 478a2886d95fe2fcaaad07790c9c422736b539ce Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 14:51:07 -0700 Subject: [PATCH 341/419] Added started tasks to progress bar --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 16390708ca..f7f5c2fb6e 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -75,11 +75,13 @@ private[spark] class IndexPage(parent: JobProgressUI) { } } - def makeProgressBar(completed: Int, total: Int): Seq[Node] = { + def makeProgressBar(started: Int, completed: Int, total: Int): Seq[Node] = { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + val startWidth = "width: %s%%".format((started.toDouble/total)*100)
    -
    +
    +
    } @@ -96,6 +98,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { case (false, true) => "Write" case _ => "" } + val startedTasks = listener.stageToTasksActive.getOrElse(s.id, Seq[Long]()).size val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions @@ -105,7 +108,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeProgressBar(completedTasks, totalTasks)} + {makeProgressBar(startedTasks, completedTasks, totalTasks)} {completedTasks} / {totalTasks} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) From 2d6da9195ab8feeaf3fc478ec09d22568d6aa64b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 15:50:52 -0700 Subject: [PATCH 342/419] Alphabetized imports --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index c12a4fe2a5..646ae5ecbc 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -25,9 +25,9 @@ import scala.Some import scala.xml.{NodeSeq, Node} import spark.scheduler.Stage -import spark.ui.UIUtils._ -import spark.ui.Page._ import spark.storage.StorageLevel +import spark.ui.Page._ +import spark.ui.UIUtils._ import spark.Utils /** Page showing list of all ongoing and recently finished stages */ From 07da72b45190f7db9daa2c6bd33577d28e19e659 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 29 Jul 2013 16:25:17 -0700 Subject: [PATCH 343/419] Remove duplicate loss history and clarify why. Also some minor style fixes. --- .../mllib/classification/LogisticRegression.scala | 5 +---- .../mllib/optimization/GradientDescent.scala | 15 +++++++-------- .../scala/spark/mllib/optimization/Updater.scala | 2 +- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index bc1c327729..bf3b05dedb 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -151,7 +151,6 @@ object LogisticRegressionLocalRandomSGD { input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, - miniBatchFraction: Double, initialWeights: Array[Double]) : LogisticRegressionModel = @@ -174,7 +173,6 @@ object LogisticRegressionLocalRandomSGD { input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, - miniBatchFraction: Double) : LogisticRegressionModel = { @@ -195,8 +193,7 @@ object LogisticRegressionLocalRandomSGD { def train( input: RDD[(Int, Array[Double])], numIterations: Int, - stepSize: Double - ) + stepSize: Double) : LogisticRegressionModel = { train(input, numIterations, stepSize, 1.0) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index d4b83a1456..19cda26446 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -61,7 +61,7 @@ object GradientDescent { // Initialize weights as a column vector var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) - var reg_val = 0.0 + var regVal = 0.0 for (i <- 1 to numIters) { val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { @@ -71,15 +71,14 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) + /** + * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration + * and regVal is the regularization value computed in the previous iteration as well. + */ + stochasticLossHistory.append(lossSum / miniBatchSize + regVal) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) weights = update._1 - reg_val = update._2 - stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) - /* - * NOTE(Xinghao): The loss here is sum of lossSum computed using the weights before applying updater, - * and reg_val using weights after applying updater - */ + regVal = update._2 } (weights.toArray, stochasticLossHistory.toArray) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index 188fe7d972..bbf21e5c28 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -76,7 +76,7 @@ class SquaredL2Updater extends Updater { val thisIterStepSize = stepSize / math.sqrt(iter) val normGradient = gradient.mul(thisIterStepSize) val newWeights = weightsOld.sub(normGradient).div(2.0 * thisIterStepSize * regParam + 1.0) - (newWeights, pow(newWeights.norm2,2.0) * regParam) + (newWeights, pow(newWeights.norm2, 2.0) * regParam) } } From 87b821dc3953bac7584592b9b030c2f57203dd0d Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 16:50:51 -0700 Subject: [PATCH 344/419] Fixed continuity of executorToTasksActive, changed color of progress bars --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index b70153fd30..d6f83157d3 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -93,7 +93,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, Seq[Long]()).size.toString + val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, HashSet[TaskInfo]()).size.toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index e0a4168290..e2cf84346a 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -21,9 +21,11 @@ import java.util.Date import javax.servlet.http.HttpServletRequest +import scala.collection.mutable.HashSet import scala.Some import scala.xml.{NodeSeq, Node} +import spark.scheduler.cluster.TaskInfo import spark.scheduler.Stage import spark.storage.StorageLevel import spark.ui.Page._ @@ -110,8 +112,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { val startWidth = "width: %s%%".format((started.toDouble/total)*100)
    -
    -
    +
    +
    } @@ -131,6 +133,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { case b => Utils.memoryBytesToString(b) } + val startedTasks = listener.stageToTasksActive.getOrElse(s.id, HashSet[TaskInfo]()).size val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions From 17e62113d446c87d0313daa2ac230867591627be Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:14:44 -0700 Subject: [PATCH 345/419] Moved DeployMessage's into its own DeployMessages object. Also renamed MasterState to MasterStateResponse and WorkerState to WorkerStateResponse for clarity. --- .../scala/spark/deploy/DeployMessage.scala | 172 +++++++++--------- .../scala/spark/deploy/JsonProtocol.scala | 11 +- .../scala/spark/deploy/client/Client.scala | 18 +- .../scala/spark/deploy/master/Master.scala | 39 ++-- .../deploy/master/ui/ApplicationPage.scala | 11 +- .../spark/deploy/master/ui/IndexPage.scala | 16 +- .../spark/deploy/worker/ExecutorRunner.scala | 10 +- .../scala/spark/deploy/worker/Worker.scala | 30 +-- .../spark/deploy/worker/ui/IndexPage.scala | 23 ++- 9 files changed, 174 insertions(+), 156 deletions(-) diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index e1f8aff6f5..7c37a16615 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -17,109 +17,107 @@ package spark.deploy +import scala.collection.immutable.List + +import spark.Utils import spark.deploy.ExecutorState.ExecutorState import spark.deploy.master.{WorkerInfo, ApplicationInfo} import spark.deploy.worker.ExecutorRunner -import scala.collection.immutable.List -import spark.Utils -private[spark] sealed trait DeployMessage extends Serializable +private[deploy] sealed trait DeployMessage extends Serializable -// Worker to Master +private[deploy] object DeployMessages { -private[spark] -case class RegisterWorker( - id: String, - host: String, - port: Int, - cores: Int, - memory: Int, - webUiPort: Int, - publicAddress: String) - extends DeployMessage { - Utils.checkHost(host, "Required hostname") - assert (port > 0) -} + // Worker to Master -private[spark] -case class ExecutorStateChanged( - appId: String, - execId: Int, - state: ExecutorState, - message: Option[String], + case class RegisterWorker( + id: String, + host: String, + port: Int, + cores: Int, + memory: Int, + webUiPort: Int, + publicAddress: String) + extends DeployMessage { + Utils.checkHost(host, "Required hostname") + assert (port > 0) + } + + case class ExecutorStateChanged( + appId: String, + execId: Int, + state: ExecutorState, + message: Option[String], + exitStatus: Option[Int]) + extends DeployMessage + + case class Heartbeat(workerId: String) extends DeployMessage + + // Master to Worker + + case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage + + case class RegisterWorkerFailed(message: String) extends DeployMessage + + case class KillExecutor(appId: String, execId: Int) extends DeployMessage + + case class LaunchExecutor( + appId: String, + execId: Int, + appDesc: ApplicationDescription, + cores: Int, + memory: Int, + sparkHome: String) + extends DeployMessage + + // Client to Master + + case class RegisterApplication(appDescription: ApplicationDescription) + extends DeployMessage + + // Master to Client + + case class RegisteredApplication(appId: String) extends DeployMessage + + case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { + Utils.checkHostPort(hostPort, "Required hostport") + } + + case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String], exitStatus: Option[Int]) - extends DeployMessage -private[spark] case class Heartbeat(workerId: String) extends DeployMessage + case class ApplicationRemoved(message: String) -// Master to Worker + // Internal message in Client -private[spark] case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage -private[spark] case class RegisterWorkerFailed(message: String) extends DeployMessage -private[spark] case class KillExecutor(appId: String, execId: Int) extends DeployMessage + case object StopClient -private[spark] case class LaunchExecutor( - appId: String, - execId: Int, - appDesc: ApplicationDescription, - cores: Int, - memory: Int, - sparkHome: String) - extends DeployMessage + // MasterWebUI To Master -// Client to Master + case object RequestMasterState -private[spark] case class RegisterApplication(appDescription: ApplicationDescription) - extends DeployMessage + // Master to MasterWebUI -// Master to Client + case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo], + activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) { -private[spark] -case class RegisteredApplication(appId: String) extends DeployMessage + Utils.checkHost(host, "Required hostname") + assert (port > 0) -private[spark] -case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { - Utils.checkHostPort(hostPort, "Required hostport") -} - -private[spark] -case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String], - exitStatus: Option[Int]) - -private[spark] -case class ApplicationRemoved(message: String) - -// Internal message in Client - -private[spark] case object StopClient - -// MasterWebUI To Master - -private[spark] case object RequestMasterState - -// Master to MasterWebUI - -private[spark] -case class MasterState(host: String, port: Int, workers: Array[WorkerInfo], - activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) { - - Utils.checkHost(host, "Required hostname") - assert (port > 0) - - def uri = "spark://" + host + ":" + port -} - -// WorkerWebUI to Worker -private[spark] case object RequestWorkerState - -// Worker to WorkerWebUI - -private[spark] -case class WorkerState(host: String, port: Int, workerId: String, executors: List[ExecutorRunner], - finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int, - coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) { - - Utils.checkHost(host, "Required hostname") - assert (port > 0) + def uri = "spark://" + host + ":" + port + } + + // WorkerWebUI to Worker + case object RequestWorkerState + + // Worker to WorkerWebUI + + case class WorkerStateResponse(host: String, port: Int, workerId: String, + executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String, + cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) { + + Utils.checkHost(host, "Required hostname") + assert (port > 0) + } } diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 64f89623e1..bd1db7c294 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -17,9 +17,12 @@ package spark.deploy -import master.{ApplicationInfo, WorkerInfo} import net.liftweb.json.JsonDSL._ -import worker.ExecutorRunner + +import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} +import spark.deploy.master.{ApplicationInfo, WorkerInfo} +import spark.deploy.worker.ExecutorRunner + private[spark] object JsonProtocol { def writeWorkerInfo(obj: WorkerInfo) = { @@ -57,7 +60,7 @@ private[spark] object JsonProtocol { ("appdesc" -> writeApplicationDescription(obj.appDesc)) } - def writeMasterState(obj: MasterState) = { + def writeMasterState(obj: MasterStateResponse) = { ("url" -> ("spark://" + obj.uri)) ~ ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ ("cores" -> obj.workers.map(_.cores).sum) ~ @@ -68,7 +71,7 @@ private[spark] object JsonProtocol { ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) } - def writeWorkerState(obj: WorkerState) = { + def writeWorkerState(obj: WorkerStateResponse) = { ("id" -> obj.workerId) ~ ("masterurl" -> obj.masterUrl) ~ ("masterwebuiurl" -> obj.masterWebUiUrl) ~ diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 29e494f495..9d5ba8a796 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -17,21 +17,23 @@ package spark.deploy.client -import spark.deploy._ +import java.util.concurrent.TimeoutException + import akka.actor._ +import akka.actor.Terminated import akka.pattern.ask import akka.util.Duration -import akka.util.duration._ -import java.util.concurrent.TimeoutException -import spark.{SparkException, Logging} +import akka.remote.RemoteClientDisconnected import akka.remote.RemoteClientLifeCycleEvent import akka.remote.RemoteClientShutdown -import spark.deploy.RegisterApplication -import spark.deploy.master.Master -import akka.remote.RemoteClientDisconnected -import akka.actor.Terminated import akka.dispatch.Await +import spark.Logging +import spark.deploy.{ApplicationDescription, ExecutorState} +import spark.deploy.DeployMessages._ +import spark.deploy.master.Master + + /** * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description, * and a listener for cluster events, and calls back the listener when various events occur. diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 9692af5295..202d5bcdb7 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -17,21 +17,22 @@ package spark.deploy.master -import akka.actor._ -import akka.actor.Terminated -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} -import akka.util.duration._ - import java.text.SimpleDateFormat import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import spark.deploy._ +import akka.actor._ +import akka.actor.Terminated +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} +import akka.util.duration._ + import spark.{Logging, SparkException, Utils} +import spark.deploy.{ApplicationDescription, ExecutorState} +import spark.deploy.DeployMessages._ +import spark.deploy.master.ui.MasterWebUI import spark.metrics.MetricsSystem import spark.util.AkkaUtils -import ui.MasterWebUI private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { @@ -168,7 +169,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } case RequestMasterState => { - sender ! MasterState(host, port, workers.toArray, apps.toArray, completedApps.toArray) + sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray) } } @@ -233,20 +234,27 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) - worker.actor ! LaunchExecutor(exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome) - exec.application.driver ! ExecutorAdded(exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) + worker.actor ! LaunchExecutor( + exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome) + exec.application.driver ! ExecutorAdded( + exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) } def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int, publicAddress: String): WorkerInfo = { - // There may be one or more refs to dead workers on this same node (w/ different ID's), remove them. - workers.filter(w => (w.host == host && w.port == port) && (w.state == WorkerState.DEAD)).foreach(workers -= _) + // There may be one or more refs to dead workers on this same node (w/ different ID's), + // remove them. + workers.filter { w => + (w.host == host && w.port == port) && (w.state == WorkerState.DEAD) + }.foreach { w => + workers -= w + } val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress) workers += worker idToWorker(worker.id) = worker actorToWorker(sender) = worker addressToWorker(sender.path.address) = worker - return worker + worker } def removeWorker(worker: WorkerInfo) { @@ -257,7 +265,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act addressToWorker -= worker.actor.path.address for (exec <- worker.executors.values) { logInfo("Telling app of lost executor: " + exec.id) - exec.application.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None) + exec.application.driver ! ExecutorUpdated( + exec.id, ExecutorState.LOST, Some("worker lost"), None) exec.application.removeExecutor(exec) } } @@ -277,7 +286,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= desc.memoryPerSlave)) { logWarning("Could not find any workers with enough memory for " + firstApp.get.id) } - return app + app } def finishApplication(app: ApplicationInfo) { diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 32264af393..b4c62bc224 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -17,6 +17,8 @@ package spark.deploy.master.ui +import scala.xml.Node + import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ @@ -25,9 +27,8 @@ import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import scala.xml.Node - -import spark.deploy.{RequestMasterState, JsonProtocol, MasterState} +import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import spark.deploy.JsonProtocol import spark.deploy.master.ExecutorInfo import spark.ui.UIUtils @@ -38,7 +39,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { /** Executor details for a particular application */ def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) @@ -49,7 +50,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { /** Executor details for a particular application */ def render(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index b05197c1b9..557df89b41 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -17,18 +17,20 @@ package spark.deploy.master.ui -import akka.dispatch.Await -import akka.pattern.ask -import akka.util.duration._ - import javax.servlet.http.HttpServletRequest import scala.xml.Node -import spark.deploy.{RequestMasterState, DeployWebUI, MasterState} +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.duration._ + import spark.Utils -import spark.ui.UIUtils +import spark.deploy.DeployWebUI +import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import spark.deploy.master.{ApplicationInfo, WorkerInfo} +import spark.ui.UIUtils + private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.master @@ -36,7 +38,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 47d3390928..345dfe879c 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -19,14 +19,12 @@ package spark.deploy.worker import java.io._ import java.lang.System.getenv -import spark.deploy.{ExecutorState, ExecutorStateChanged, ApplicationDescription} + import akka.actor.ActorRef + import spark.{Utils, Logging} -import java.net.{URI, URL} -import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.hadoop.conf.Configuration -import scala.Some -import spark.deploy.ExecutorStateChanged +import spark.deploy.{ExecutorState, ApplicationDescription} +import spark.deploy.DeployMessages.ExecutorStateChanged /** * Manages the execution of one executor process. diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 8fa0d12b82..0e46fa281e 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -17,22 +17,24 @@ package spark.deploy.worker -import scala.collection.mutable.{ArrayBuffer, HashMap} -import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import akka.util.duration._ -import spark.{Logging, Utils} -import spark.util.AkkaUtils -import spark.deploy._ -import spark.metrics.MetricsSystem -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.text.SimpleDateFormat import java.util.Date -import spark.deploy.RegisterWorker -import spark.deploy.LaunchExecutor -import spark.deploy.RegisterWorkerFailed -import spark.deploy.master.Master import java.io.File -import ui.WorkerWebUI + +import scala.collection.mutable.HashMap + +import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} +import akka.util.duration._ + +import spark.{Logging, Utils} +import spark.deploy.ExecutorState +import spark.deploy.DeployMessages._ +import spark.deploy.master.Master +import spark.deploy.worker.ui.WorkerWebUI +import spark.metrics.MetricsSystem +import spark.util.AkkaUtils + private[spark] class Worker( host: String, @@ -164,7 +166,7 @@ private[spark] class Worker( masterDisconnected() case RequestWorkerState => { - sender ! WorkerState(host, port, workerId, executors.values.toList, + sender ! WorkerStateResponse(host, port, workerId, executors.values.toList, finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed, masterWebUiUrl) } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 7548a26c2e..1619c6a4c2 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -17,34 +17,36 @@ package spark.deploy.worker.ui +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ -import javax.servlet.http.HttpServletRequest - import net.liftweb.json.JsonAST.JValue -import scala.xml.Node - -import spark.deploy.{RequestWorkerState, JsonProtocol, WorkerState} -import spark.deploy.worker.ExecutorRunner import spark.Utils +import spark.deploy.JsonProtocol +import spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} +import spark.deploy.worker.ExecutorRunner import spark.ui.UIUtils + private[spark] class IndexPage(parent: WorkerWebUI) { val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout def renderJson(request: HttpServletRequest): JValue = { - val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, 30 seconds) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, 30 seconds) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") @@ -69,7 +71,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
    -
    +
    @@ -88,7 +90,8 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
    ; - UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format( + workerState.host, workerState.port)) } def executorRow(executor: ExecutorRunner): Seq[Node] = { From 207548b67bc6ce208b60215d40a03904d8ac0cbc Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 17:19:33 -0700 Subject: [PATCH 346/419] Open up Job UI ports (33000-33010) on EC2 clusters --- ec2/spark_ec2.py | 39 ++++++++++++++++++++------------------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 2ec3c007fb..740ec08542 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -9,9 +9,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -53,7 +53,7 @@ def parse_args(): help="Seconds to wait for nodes to start (default: 120)") parser.add_option("-k", "--key-pair", help="Key pair to use on instances") - parser.add_option("-i", "--identity-file", + parser.add_option("-i", "--identity-file", help="SSH private key file to use for logging into instances") parser.add_option("-t", "--instance-type", default="m1.large", help="Type of instance to launch (default: m1.large). " + @@ -69,7 +69,7 @@ def parse_args(): parser.add_option("-a", "--ami", default="latest", help="Amazon Machine Image ID to use, or 'latest' to use latest " + "available AMI (default: latest)") - parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port", + parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port", help="Use SSH dynamic port forwarding to create a SOCKS proxy at " + "the given local address (for use with login)") parser.add_option("--resume", action="store_true", default=False, @@ -99,7 +99,7 @@ def parse_args(): help="The SSH user you want to connect as (default: root)") parser.add_option("--delete-groups", action="store_true", default=False, help="When destroying a cluster, delete the security groups that were created") - + (opts, args) = parser.parse_args() if len(args) != 2: parser.print_help() @@ -112,7 +112,7 @@ def parse_args(): if opts.cluster_type not in ["mesos", "standalone"] and action == "launch": print >> stderr, ("ERROR: Invalid cluster type: " + opts.cluster_type) sys.exit(1) - + # Boto config check # http://boto.cloudhackers.com/en/latest/boto_config_tut.html home_dir = os.getenv('HOME') @@ -178,6 +178,7 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') + master_group.authorize('tcp', 33000, 33010, '0.0.0.0/0') if opts.cluster_type == "mesos": master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0') if opts.ganglia: @@ -257,7 +258,7 @@ def launch_cluster(conn, opts, cluster_name): block_device_map = block_map) my_req_ids += [req.id for req in slave_reqs] i += 1 - + print "Waiting for spot instances to be granted..." try: while True: @@ -413,7 +414,7 @@ def setup_standalone_cluster(master, slave_nodes, opts): slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes]) ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips)) ssh(master, opts, "/root/spark/bin/start-all.sh") - + def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") @@ -528,7 +529,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes, dest.write(text) dest.close() # rsync the whole directory over to the master machine - command = (("rsync -rv -e 'ssh -o StrictHostKeyChecking=no -i %s' " + + 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) # Remove the temp directory we created above @@ -557,9 +558,9 @@ def ssh(host, opts, command): print "Error connecting to host {0}, sleeping 30".format(e) time.sleep(30) tries = tries + 1 - - - + + + # Gets a list of zones to launch instances in @@ -618,12 +619,12 @@ def main(): print "Terminating zoo..." for inst in zoo_nodes: inst.terminate() - + # Delete security groups as well if opts.delete_groups: print "Deleting security groups (this will take some time)..." group_names = [cluster_name + "-master", cluster_name + "-slaves", cluster_name + "-zoo"] - + attempt = 1; while attempt <= 3: print "Attempt %d" % attempt @@ -639,7 +640,7 @@ def main(): from_port=rule.from_port, to_port=rule.to_port, src_group=grant) - + # Sleep for AWS eventual-consistency to catch up, and for instances # to terminate time.sleep(30) # Yes, it does have to be this long :-( @@ -650,13 +651,13 @@ def main(): except boto.exception.EC2ResponseError: success = False; print "Failed to delete security group " + group.name - + # Unfortunately, group.revoke() returns True even if a rule was not # deleted, so this needs to be rerun if something fails if success: break; - + attempt += 1 - + if not success: print "Failed to delete all security groups after 3 tries." print "Try re-running in a few minutes." @@ -679,7 +680,7 @@ def main(): elif action == "stop": response = raw_input("Are you sure you want to stop the cluster " + cluster_name + "?\nDATA ON EPHEMERAL DISKS WILL BE LOST, " + - "BUT THE CLUSTER WILL KEEP USING SPACE ON\n" + + "BUT THE CLUSTER WILL KEEP USING SPACE ON\n" + "AMAZON EBS IF IT IS EBS-BACKED!!\n" + "Stop cluster " + cluster_name + " (y/N): ") if response == "y": From 105f4d22e93af23ff3bf117bcfd976d412f494e6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:30:38 -0700 Subject: [PATCH 347/419] Removed Cache and SoftReferenceCache since they are no longer used. --- core/src/main/scala/spark/Cache.scala | 80 ------------------- .../main/scala/spark/SoftReferenceCache.scala | 35 -------- 2 files changed, 115 deletions(-) delete mode 100644 core/src/main/scala/spark/Cache.scala delete mode 100644 core/src/main/scala/spark/SoftReferenceCache.scala diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala deleted file mode 100644 index b0c83ce59d..0000000000 --- a/core/src/main/scala/spark/Cache.scala +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package spark - -import java.util.concurrent.atomic.AtomicInteger - -private[spark] sealed trait CachePutResponse -private[spark] case class CachePutSuccess(size: Long) extends CachePutResponse -private[spark] case class CachePutFailure() extends CachePutResponse - -/** - * An interface for caches in Spark, to allow for multiple implementations. Caches are used to store - * both partitions of cached RDDs and broadcast variables on Spark executors. Caches are also aware - * of which entries are part of the same dataset (for example, partitions in the same RDD). The key - * for each value in a cache is a (datasetID, partition) pair. - * - * A single Cache instance gets created on each machine and is shared by all caches (i.e. both the - * RDD split cache and the broadcast variable cache), to enable global replacement policies. - * However, because these several independent modules all perform caching, it is important to give - * them separate key namespaces, so that an RDD and a broadcast variable (for example) do not use - * the same key. For this purpose, Cache has the notion of KeySpaces. Each client module must first - * ask for a KeySpace, and then call get() and put() on that space using its own keys. - * - * This abstract class handles the creation of key spaces, so that subclasses need only deal with - * keys that are unique across modules. - */ -private[spark] abstract class Cache { - private val nextKeySpaceId = new AtomicInteger(0) - private def newKeySpaceId() = nextKeySpaceId.getAndIncrement() - - def newKeySpace() = new KeySpace(this, newKeySpaceId()) - - /** - * Get the value for a given (datasetId, partition), or null if it is not - * found. - */ - def get(datasetId: Any, partition: Int): Any - - /** - * Attempt to put a value in the cache; returns CachePutFailure if this was - * not successful (e.g. because the cache replacement policy forbids it), and - * CachePutSuccess if successful. If size estimation is available, the cache - * implementation should set the size field in CachePutSuccess. - */ - def put(datasetId: Any, partition: Int, value: Any): CachePutResponse - - /** - * Report the capacity of the cache partition. By default this just reports - * zero. Specific implementations can choose to provide the capacity number. - */ - def getCapacity: Long = 0L -} - -/** - * A key namespace in a Cache. - */ -private[spark] class KeySpace(cache: Cache, val keySpaceId: Int) { - def get(datasetId: Any, partition: Int): Any = - cache.get((keySpaceId, datasetId), partition) - - def put(datasetId: Any, partition: Int, value: Any): CachePutResponse = - cache.put((keySpaceId, datasetId), partition, value) - - def getCapacity: Long = cache.getCapacity -} diff --git a/core/src/main/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala deleted file mode 100644 index f41a379582..0000000000 --- a/core/src/main/scala/spark/SoftReferenceCache.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package spark - -import com.google.common.collect.MapMaker - -/** - * An implementation of Cache that uses soft references. - */ -private[spark] class SoftReferenceCache extends Cache { - val map = new MapMaker().softValues().makeMap[Any, Any]() - - override def get(datasetId: Any, partition: Int): Any = - map.get((datasetId, partition)) - - override def put(datasetId: Any, partition: Int, value: Any): CachePutResponse = { - map.put((datasetId, partition), value) - return CachePutSuccess(0) - } -} From 23b5da14ed6413b0dcb4c0bfdb80c98c433f3c9d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:42:05 -0700 Subject: [PATCH 348/419] Moved block manager messages into BlockManagerMessages object. --- .../spark/storage/BlockManagerMaster.scala | 1 + .../storage/BlockManagerMasterActor.scala | 2 + .../spark/storage/BlockManagerMessages.scala | 159 ++++++++---------- .../storage/BlockManagerSlaveActor.scala | 2 +- .../spark/storage/BlockManagerSource.scala | 2 +- .../scala/spark/storage/BlockMessage.scala | 1 - .../spark/storage/BlockMessageArray.scala | 5 +- .../spark/storage/BlockObjectWriter.scala | 2 - 8 files changed, 80 insertions(+), 94 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 3186f7c85b..76128e8cff 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -23,6 +23,7 @@ import akka.pattern.ask import akka.util.Duration import spark.{Logging, SparkException} +import spark.storage.BlockManagerMessages._ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 244000d952..011bb6b83d 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -29,6 +29,8 @@ import akka.util.Duration import akka.util.duration._ import spark.{Logging, Utils, SparkException} +import spark.storage.BlockManagerMessages._ + /** * BlockManagerMasterActor is an actor on the master node to track statuses of diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 01de4ccb8f..9375a9ca54 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -22,102 +22,89 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import akka.actor.ActorRef -////////////////////////////////////////////////////////////////////////////////// -// Messages from the master to slaves. -////////////////////////////////////////////////////////////////////////////////// -private[spark] -sealed trait ToBlockManagerSlave +private[storage] object BlockManagerMessages { + ////////////////////////////////////////////////////////////////////////////////// + // Messages from the master to slaves. + ////////////////////////////////////////////////////////////////////////////////// + sealed trait ToBlockManagerSlave -// Remove a block from the slaves that have it. This can only be used to remove -// blocks that the master knows about. -private[spark] -case class RemoveBlock(blockId: String) extends ToBlockManagerSlave + // Remove a block from the slaves that have it. This can only be used to remove + // blocks that the master knows about. + case class RemoveBlock(blockId: String) extends ToBlockManagerSlave -// Remove all blocks belonging to a specific RDD. -private[spark] case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave + // Remove all blocks belonging to a specific RDD. + case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave -////////////////////////////////////////////////////////////////////////////////// -// Messages from slaves to the master. -////////////////////////////////////////////////////////////////////////////////// -private[spark] -sealed trait ToBlockManagerMaster + ////////////////////////////////////////////////////////////////////////////////// + // Messages from slaves to the master. + ////////////////////////////////////////////////////////////////////////////////// + sealed trait ToBlockManagerMaster -private[spark] -case class RegisterBlockManager( - blockManagerId: BlockManagerId, - maxMemSize: Long, - sender: ActorRef) - extends ToBlockManagerMaster + case class RegisterBlockManager( + blockManagerId: BlockManagerId, + maxMemSize: Long, + sender: ActorRef) + extends ToBlockManagerMaster -private[spark] -case class HeartBeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster + case class HeartBeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster -private[spark] -class UpdateBlockInfo( - var blockManagerId: BlockManagerId, - var blockId: String, - var storageLevel: StorageLevel, - var memSize: Long, - var diskSize: Long) - extends ToBlockManagerMaster - with Externalizable { + class UpdateBlockInfo( + var blockManagerId: BlockManagerId, + var blockId: String, + var storageLevel: StorageLevel, + var memSize: Long, + var diskSize: Long) + extends ToBlockManagerMaster + with Externalizable { - def this() = this(null, null, null, 0, 0) // For deserialization only + def this() = this(null, null, null, 0, 0) // For deserialization only - override def writeExternal(out: ObjectOutput) { - blockManagerId.writeExternal(out) - out.writeUTF(blockId) - storageLevel.writeExternal(out) - out.writeLong(memSize) - out.writeLong(diskSize) + override def writeExternal(out: ObjectOutput) { + blockManagerId.writeExternal(out) + out.writeUTF(blockId) + storageLevel.writeExternal(out) + out.writeLong(memSize) + out.writeLong(diskSize) + } + + override def readExternal(in: ObjectInput) { + blockManagerId = BlockManagerId(in) + blockId = in.readUTF() + storageLevel = StorageLevel(in) + memSize = in.readLong() + diskSize = in.readLong() + } } - override def readExternal(in: ObjectInput) { - blockManagerId = BlockManagerId(in) - blockId = in.readUTF() - storageLevel = StorageLevel(in) - memSize = in.readLong() - diskSize = in.readLong() + object UpdateBlockInfo { + def apply(blockManagerId: BlockManagerId, + blockId: String, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long): UpdateBlockInfo = { + new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) + } + + // For pattern-matching + def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = { + Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) + } } + + case class GetLocations(blockId: String) extends ToBlockManagerMaster + + case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster + + case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster + + case class RemoveExecutor(execId: String) extends ToBlockManagerMaster + + case object StopBlockManagerMaster extends ToBlockManagerMaster + + case object GetMemoryStatus extends ToBlockManagerMaster + + case object ExpireDeadHosts extends ToBlockManagerMaster + + case object GetStorageStatus extends ToBlockManagerMaster } - -private[spark] -object UpdateBlockInfo { - def apply(blockManagerId: BlockManagerId, - blockId: String, - storageLevel: StorageLevel, - memSize: Long, - diskSize: Long): UpdateBlockInfo = { - new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) - } - - // For pattern-matching - def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) - } -} - -private[spark] -case class GetLocations(blockId: String) extends ToBlockManagerMaster - -private[spark] -case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster - -private[spark] -case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster - -private[spark] -case class RemoveExecutor(execId: String) extends ToBlockManagerMaster - -private[spark] -case object StopBlockManagerMaster extends ToBlockManagerMaster - -private[spark] -case object GetMemoryStatus extends ToBlockManagerMaster - -private[spark] -case object ExpireDeadHosts extends ToBlockManagerMaster - -private[spark] -case object GetStorageStatus extends ToBlockManagerMaster diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala index 45cffad810..6e5fb43732 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala @@ -19,7 +19,7 @@ package spark.storage import akka.actor.Actor -import spark.{Logging, SparkException, Utils} +import spark.storage.BlockManagerMessages._ /** diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index 4faa715c94..2aecd1ea71 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -3,7 +3,7 @@ package spark.storage import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -import spark.storage._ + private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala index ab72dbb62b..bcce26b7c1 100644 --- a/core/src/main/scala/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/spark/storage/BlockMessage.scala @@ -22,7 +22,6 @@ import java.nio.ByteBuffer import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer -import spark._ import spark.network._ private[spark] case class GetBlock(id: String) diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala index b0229d6124..ee2fc167d5 100644 --- a/core/src/main/scala/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala @@ -19,7 +19,6 @@ package spark.storage import java.nio.ByteBuffer -import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer import spark._ @@ -113,7 +112,7 @@ private[spark] object BlockMessageArray { def main(args: Array[String]) { val blockMessages = - (0 until 10).map(i => { + (0 until 10).map { i => if (i % 2 == 0) { val buffer = ByteBuffer.allocate(100) buffer.clear @@ -121,7 +120,7 @@ private[spark] object BlockMessageArray { } else { BlockMessage.fromGetBlock(GetBlock(i.toString)) } - }) + } val blockMessageArray = new BlockMessageArray(blockMessages) println("Block message array created") diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala index 01ed6e8c1f..3812009ca1 100644 --- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -17,8 +17,6 @@ package spark.storage -import java.nio.ByteBuffer - /** * An interface for writing JVM objects to some underlying storage. This interface allows From 81720e13fc9e1f475dd1333babfa08f3f806a5d0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:53:01 -0700 Subject: [PATCH 349/419] Moved all StandaloneClusterMessage's into StandaloneClusterMessages object. --- .../executor/StandaloneExecutorBackend.scala | 17 ++--- .../cluster/StandaloneClusterMessage.scala | 65 ++++++++++--------- .../cluster/StandaloneSchedulerBackend.scala | 9 +-- 3 files changed, 45 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index f4003da732..e47fe50021 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -18,19 +18,16 @@ package spark.executor import java.nio.ByteBuffer -import spark.Logging -import spark.TaskState.TaskState -import spark.util.AkkaUtils + import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} -import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} -import spark.scheduler.cluster._ -import spark.scheduler.cluster.RegisteredExecutor -import spark.scheduler.cluster.LaunchTask -import spark.scheduler.cluster.RegisterExecutorFailed -import spark.scheduler.cluster.RegisterExecutor -import spark.Utils + +import spark.{Logging, Utils} +import spark.TaskState.TaskState import spark.deploy.SparkHadoopUtil +import spark.scheduler.cluster.StandaloneClusterMessages._ +import spark.util.AkkaUtils + private[spark] class StandaloneExecutorBackend( driverUrl: String, diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index ac9e5ef94d..05c29eb72f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -17,46 +17,47 @@ package spark.scheduler.cluster -import spark.TaskState.TaskState import java.nio.ByteBuffer -import spark.util.SerializableBuffer + +import spark.TaskState.TaskState import spark.Utils +import spark.util.SerializableBuffer + private[spark] sealed trait StandaloneClusterMessage extends Serializable -// Driver to executors -private[spark] -case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage +private[spark] object StandaloneClusterMessages { -private[spark] -case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) - extends StandaloneClusterMessage + // Driver to executors + case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage -private[spark] -case class RegisterExecutorFailed(message: String) extends StandaloneClusterMessage + case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) + extends StandaloneClusterMessage -// Executors to driver -private[spark] -case class RegisterExecutor(executorId: String, hostPort: String, cores: Int) - extends StandaloneClusterMessage { - Utils.checkHostPort(hostPort, "Expected host port") -} + case class RegisterExecutorFailed(message: String) extends StandaloneClusterMessage -private[spark] -case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer) - extends StandaloneClusterMessage - -private[spark] -object StatusUpdate { - /** Alternate factory method that takes a ByteBuffer directly for the data field */ - def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer): StatusUpdate = { - StatusUpdate(executorId, taskId, state, new SerializableBuffer(data)) + // Executors to driver + case class RegisterExecutor(executorId: String, hostPort: String, cores: Int) + extends StandaloneClusterMessage { + Utils.checkHostPort(hostPort, "Expected host port") } + + case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, + data: SerializableBuffer) extends StandaloneClusterMessage + + object StatusUpdate { + /** Alternate factory method that takes a ByteBuffer directly for the data field */ + def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer) + : StatusUpdate = { + StatusUpdate(executorId, taskId, state, new SerializableBuffer(data)) + } + } + + // Internal messages in driver + case object ReviveOffers extends StandaloneClusterMessage + + case object StopDriver extends StandaloneClusterMessage + + case class RemoveExecutor(executorId: String, reason: String) extends StandaloneClusterMessage + } - -// Internal messages in driver -private[spark] case object ReviveOffers extends StandaloneClusterMessage -private[spark] case object StopDriver extends StandaloneClusterMessage - -private[spark] case class RemoveExecutor(executorId: String, reason: String) - extends StandaloneClusterMessage diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 03a64e0192..075a7cbf7e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -17,17 +17,18 @@ package spark.scheduler.cluster +import java.util.concurrent.atomic.AtomicInteger + import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import akka.actor._ -import akka.util.duration._ +import akka.dispatch.Await import akka.pattern.ask +import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} import akka.util.Duration import spark.{Utils, SparkException, Logging, TaskState} -import akka.dispatch.Await -import java.util.concurrent.atomic.AtomicInteger -import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import spark.scheduler.cluster.StandaloneClusterMessages._ /** * A standalone scheduler backend, which waits for standalone executors to connect to it through From 3ca9faa341dcddb54f8b2e26b582c08901ea875f Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 29 Jul 2013 18:37:28 -0700 Subject: [PATCH 350/419] Clarify how regVal is computed in Updater docs --- .../spark/mllib/optimization/Updater.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index bbf21e5c28..e916a92c33 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -23,6 +23,7 @@ import org.jblas.DoubleMatrix abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize and iteration number. + * Also returns the regularization value computed using the *updated* weights. * * @param weightsOlds - Column matrix of size nx1 where n is the number of features. * @param gradient - Column matrix of size nx1 where n is the number of features. @@ -31,7 +32,7 @@ abstract class Updater extends Serializable { * @param regParam - Regularization parameter * * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, - * and the second element is the regularization value. + * and the second element is the regularization value computed using updated weights. */ def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) @@ -46,13 +47,13 @@ class SimpleUpdater extends Updater { } /** -* L1 regularization -- corresponding proximal operator is the soft-thresholding function -* That is, each weight component is shrunk towards 0 by shrinkageVal -* If w > shrinkageVal, set weight component to w-shrinkageVal. -* If w < -shrinkageVal, set weight component to w+shrinkageVal. -* If -shrinkageVal < w < shrinkageVal, set weight component to 0. -* Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) -**/ + * L1 regularization -- corresponding proximal operator is the soft-thresholding function + * That is, each weight component is shrunk towards 0 by shrinkageVal + * If w > shrinkageVal, set weight component to w-shrinkageVal. + * If w < -shrinkageVal, set weight component to w+shrinkageVal. + * If -shrinkageVal < w < shrinkageVal, set weight component to 0. + * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) + */ class L1Updater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { From 614ee16cc4c63260f13d0c7494fbaafa8a061e95 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 30 Jul 2013 10:57:26 +0800 Subject: [PATCH 351/419] refactor job ui with pool information --- core/src/main/scala/spark/SparkContext.scala | 16 +++-- .../scala/spark/scheduler/SparkListener.scala | 2 +- .../scheduler/cluster/ClusterScheduler.scala | 12 ++-- .../cluster/ClusterTaskSetManager.scala | 7 +-- .../spark/scheduler/cluster/Schedulable.scala | 2 +- .../cluster/SchedulableBuilder.scala | 6 +- .../scheduler/cluster/SchedulingMode.scala | 6 +- .../scheduler/cluster/TaskSetManager.scala | 1 + .../scheduler/local/LocalScheduler.scala | 5 +- .../scheduler/local/LocalTaskSetManager.scala | 7 +-- .../scala/spark/ui/UIWorkloadGenerator.scala | 35 ++++++++--- .../main/scala/spark/ui/jobs/IndexPage.scala | 28 +++------ .../spark/ui/jobs/JobProgressListener.scala | 62 +++++-------------- .../scala/spark/ui/jobs/JobProgressUI.scala | 7 +-- .../main/scala/spark/ui/jobs/PoolPage.scala | 17 ++--- .../main/scala/spark/ui/jobs/PoolTable.scala | 23 +++---- .../main/scala/spark/ui/jobs/StageTable.scala | 16 ++--- 17 files changed, 116 insertions(+), 136 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index b5225d5681..375636071d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -45,13 +45,13 @@ import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} -import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable, +SchedulingMode} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} -import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -542,17 +542,25 @@ class SparkContext( env.blockManager.master.getStorageStatus } - def getPoolsInfo: ArrayBuffer[Schedulable] = { + /** + * Return pools for fair scheduler + * TODO:now, we have not taken nested pools into account + */ + def getPools: ArrayBuffer[Schedulable] = { taskScheduler.rootPool.schedulableQueue } - def getSchedulingMode: SchedulingMode = { + /** + * Return current scheduling mode + */ + def getSchedulingMode: SchedulingMode.SchedulingMode = { taskScheduler.schedulingMode } def getPoolNameToPool: HashMap[String, Schedulable] = { taskScheduler.rootPool.schedulableNameToSchedulable } + /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 94fdad9b98..07372ee786 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -8,7 +8,7 @@ import spark.executor.TaskMetrics sealed trait SparkListenerEvents -case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties = null) extends SparkListenerEvents +case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties) extends SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 1b23fd6cef..74b3e43d2b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -98,8 +98,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null - //default scheduler is FIFO - val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) + // default scheduler is FIFO + val schedulingMode: SchedulingMode = SchedulingMode.withName( + System.getProperty("spark.cluster.schedulingmode", "FIFO")) override def setListener(listener: TaskSchedulerListener) { this.listener = listener @@ -107,7 +108,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def initialize(context: SchedulerBackend) { backend = context - //temporarily set rootPool name to empty + // temporarily set rootPool name to empty rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { @@ -254,10 +255,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } var launchedTask = false val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() - for (manager <- sortedTaskSetQueue) - { + + for (manager <- sortedTaskSetQueue) { logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) } + for (manager <- sortedTaskSetQueue) { // Split offers based on node local, rack local and off-rack tasks. diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 7a6a6b7826..4d11b0959a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -91,7 +91,6 @@ private[spark] class ClusterTaskSetManager( var stageId = taskSet.stageId var name = "TaskSet_"+taskSet.stageId.toString var parent: Schedulable = null - var schedulableQueue :ArrayBuffer[Schedulable] = null // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis @@ -645,17 +644,17 @@ private[spark] class ClusterTaskSetManager( } } - //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + // TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed override def getSchedulableByName(name: String): Schedulable = { return null } override def addSchedulable(schedulable:Schedulable) { - //nothing + // nothing } override def removeSchedulable(schedulable:Schedulable) { - //nothing + // nothing } override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 2e4f14c11f..c410af8af4 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -9,7 +9,7 @@ import scala.collection.mutable.ArrayBuffer */ private[spark] trait Schedulable { var parent: Schedulable - //childrens + // child queues def schedulableQueue: ArrayBuffer[Schedulable] def schedulingMode: SchedulingMode def weight: Int diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index 18cc15c2a5..a2fa80aa36 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -27,7 +27,7 @@ private[spark] trait SchedulableBuilder { private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { override def buildPools() { - //nothing + // nothing } override def addTaskSetManager(manager: Schedulable, properties: Properties) { @@ -86,7 +86,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula } } - //finally create "default" pool + // finally create "default" pool if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) { val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) rootPool.addSchedulable(pool) @@ -102,7 +102,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula poolName = properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) parentPool = rootPool.getSchedulableByName(poolName) if (parentPool == null) { - //we will create a new pool that user has configured in app instead of being defined in xml file + // we will create a new pool that user has configured in app instead of being defined in xml file parentPool = new Pool(poolName,DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) rootPool.addSchedulable(parentPool) logInfo("Create pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index c5c7ee3b22..a7f0f6f393 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,6 +1,10 @@ package spark.scheduler.cluster -object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE"){ +/** + * "FAIR" and "FIFO" determines which policy is used to order tasks amongst a Schedulable's sub-queues + * "NONE" is used when the a Schedulable has no sub-queues. + */ +object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE") { type SchedulingMode = Value val FAIR,FIFO,NONE = Value diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 472e01b227..4e6bc51278 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -6,6 +6,7 @@ import spark.TaskState.TaskState import java.nio.ByteBuffer private[spark] trait TaskSetManager extends Schedulable { + def schedulableQueue = null def schedulingMode = SchedulingMode.NONE def taskSet: TaskSet def slaveOffer(execId: String, hostPort: String, availableCpus: Double, diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 19a48895e3..f4411582f1 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -64,7 +64,8 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null - val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) + val schedulingMode: SchedulingMode = SchedulingMode.withName( + System.getProperty("spark.cluster.schedulingmode", "FIFO")) val activeTaskSets = new HashMap[String, TaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -72,7 +73,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var localActor: ActorRef = null override def start() { - //temporarily set rootPool name to empty + // temporarily set rootPool name to empty rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index 8954f40ea9..cc27f1ecca 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -14,7 +14,6 @@ import spark.scheduler.cluster._ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { var parent: Schedulable = null - var schedulableQueue :ArrayBuffer[Schedulable] = null var weight: Int = 1 var minShare: Int = 0 var runningTasks: Int = 0 @@ -48,11 +47,11 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } def addSchedulable(schedulable: Schedulable): Unit = { - //nothing + // nothing } def removeSchedulable(schedulable: Schedulable): Unit = { - //nothing + // nothing } def getSchedulableByName(name: String): Schedulable = { @@ -60,7 +59,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } def executorLost(executorId: String, host: String): Unit = { - //nothing + // nothing } def checkSpeculatableTasks(): Boolean = { diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 8bbc6ce88e..840ac9773e 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -4,7 +4,8 @@ import scala.util.Random import spark.SparkContext import spark.SparkContext._ - +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Continuously generates jobs that expose various features of the WebUI (internal testing tool). * @@ -15,8 +16,17 @@ private[spark] object UIWorkloadGenerator { val INTER_JOB_WAIT_MS = 500 def main(args: Array[String]) { + if (args.length < 2) { + println("usage: ./run spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") + System.exit(1) + } val master = args(0) + val schedulingMode = SchedulingMode.withName(args(1)) val appName = "Spark UI Tester" + + if (schedulingMode == SchedulingMode.FAIR) { + System.setProperty("spark.cluster.schedulingmode", "FAIR") + } val sc = new SparkContext(master, appName) // NOTE: Right now there is no easy way for us to show spark.job.annotation for a given phase, @@ -56,14 +66,21 @@ private[spark] object UIWorkloadGenerator { while (true) { for ((desc, job) <- jobs) { - try { - setName(desc) - job() - println("Job funished: " + desc) - } catch { - case e: Exception => - println("Job Failed: " + desc) - } + new Thread { + override def run() { + if(schedulingMode == SchedulingMode.FAIR) { + sc.addLocalProperties("spark.scheduler.cluster.fair.pool",desc) + } + try { + setName(desc) + job() + println("Job funished: " + desc) + } catch { + case e: Exception => + println("Job Failed: " + desc) + } + } + }.start Thread.sleep(INTER_JOB_WAIT_MS) } } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index e765cecb01..abef683791 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -4,38 +4,30 @@ import java.util.Date import javax.servlet.http.HttpServletRequest -import scala.Some -import scala.xml.{NodeSeq, Node} import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +import scala.Some +import scala.xml.{NodeSeq, Node} import spark.scheduler.Stage +import spark.storage.StorageLevel import spark.ui.UIUtils._ import spark.ui.Page._ -import spark.storage.StorageLevel -import spark.scheduler.cluster.Schedulable -import spark.scheduler.cluster.SchedulingMode -import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** Page showing list of all ongoing and recently finished stages and pools*/ private[spark] class IndexPage(parent: JobProgressUI) { def listener = parent.listener - def stageTable: StageTable = parent.stageTable - - def poolTable: PoolTable = parent.poolTable - def render(request: HttpServletRequest): Seq[Node] = { val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - stageTable.setStagePoolInfo(parent.stagePoolInfo) - poolTable.setPoolSource(parent.stagePagePoolSource) + val activeStagesTable = new StageTable(activeStages, parent) + val completedStagesTable = new StageTable(completedStages, parent) + val failedStagesTable = new StageTable(failedStages, parent) - val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) - val completedStageNodeSeq = stageTable.toNodeSeq(completedStages) - val failedStageNodeSeq = stageTable.toNodeSeq(failedStages) + val poolTable = new PoolTable(parent.stagePagePoolSource, listener) val content =
    @@ -48,9 +40,9 @@ private[spark] class IndexPage(parent: JobProgressUI) {
    ++

    Pools

    ++ poolTable.toNodeSeq ++ -

    Active Stages : {activeStages.size}

    ++ activeStageNodeSeq++ -

    Completed Stages : {completedStages.size}

    ++ completedStageNodeSeq++ -

    Failed Stages : {failedStages.size}

    ++ failedStageNodeSeq +

    Active Stages : {activeStages.size}

    ++ activeStagesTable.toNodeSeq++ +

    Completed Stages : {completedStages.size}

    ++ completedStagesTable.toNodeSeq++ +

    Failed Stages : {failedStages.size}

    ++ failedStagesTable.toNodeSeq headerSparkPage(content, parent.sc, "Spark Stages/Pools", Jobs) } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala index 1244f9538b..d4767bea22 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala @@ -9,53 +9,13 @@ import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics import collection.mutable -private[spark] class FairJobProgressListener(val sparkContext: SparkContext) - extends JobProgressListener(sparkContext) { - - val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" - val DEFAULT_POOL_NAME = "default" - - override val stageToPool = HashMap[Stage, String]() - override val poolToActiveStages = HashMap[String, HashSet[Stage]]() - - override def onStageCompleted(stageCompleted: StageCompleted) = { - super.onStageCompleted(stageCompleted) - val stage = stageCompleted.stageInfo.stage - poolToActiveStages(stageToPool(stage)) -= stage - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { - super.onStageSubmitted(stageSubmitted) - val stage = stageSubmitted.stage - var poolName = DEFAULT_POOL_NAME - if (stageSubmitted.properties != null) { - poolName = stageSubmitted.properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) - } - stageToPool(stage) = poolName - val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) - stages += stage - } - - override def onJobEnd(jobEnd: SparkListenerJobEnd) { - super.onJobEnd(jobEnd) - jobEnd match { - case end: SparkListenerJobEnd => - end.jobResult match { - case JobFailed(ex, Some(stage)) => - poolToActiveStages(stageToPool(stage)) -= stage - case _ => - } - case _ => - } - } -} - private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { // How many stages to remember val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt + val DEFAULT_POOL_NAME = "default" - def stageToPool: HashMap[Stage, String] = null - def poolToActiveStages: HashMap[String, HashSet[Stage]] =null + val stageToPool = new HashMap[Stage, String]() + val poolToActiveStages = new HashMap[String, HashSet[Stage]]() val activeStages = HashSet[Stage]() val completedStages = ListBuffer[Stage]() @@ -70,6 +30,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList override def onStageCompleted(stageCompleted: StageCompleted) = { val stage = stageCompleted.stageInfo.stage + poolToActiveStages(stageToPool(stage)) -= stage activeStages -= stage completedStages += stage trimIfNecessary(completedStages) @@ -86,8 +47,18 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } } - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = - activeStages += stageSubmitted.stage + /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { + val stage = stageSubmitted.stage + activeStages += stage + var poolName = DEFAULT_POOL_NAME + if (stageSubmitted.properties != null) { + poolName = stageSubmitted.properties.getProperty("spark.scheduler.cluster.fair.pool", DEFAULT_POOL_NAME) + } + stageToPool(stage) = poolName + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) + stages += stage + } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId @@ -112,6 +83,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList end.jobResult match { case JobFailed(ex, Some(stage)) => activeStages -= stage + poolToActiveStages(stageToPool(stage)) -= stage failedStages += stage trimIfNecessary(failedStages) case _ => diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index e610252242..5703b146df 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -29,26 +29,21 @@ private[spark] class JobProgressUI(val sc: SparkContext) { private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) - var stageTable: StageTable = null var stagePoolInfo: StagePoolInfo = null - var poolTable: PoolTable = null var stagePagePoolSource: PoolSource = null def start() { + _listener = Some(new JobProgressListener(sc)) sc.getSchedulingMode match { case SchedulingMode.FIFO => - _listener = Some(new JobProgressListener(sc)) stagePoolInfo = new FIFOStagePoolInfo() stagePagePoolSource = new FIFOSource() case SchedulingMode.FAIR => - _listener = Some(new FairJobProgressListener(sc)) stagePoolInfo = new FairStagePoolInfo(listener) stagePagePoolSource = new FairSource(sc) } sc.addSparkListener(listener) - stageTable = new StageTable(dateFmt, formatDuration, listener) - poolTable = new PoolTable(listener) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala index 00703887c3..37d4f8fa6b 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -9,30 +9,23 @@ import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ -/** Page showing specific pool details*/ +/** Page showing specific pool details */ private[spark] class PoolPage(parent: JobProgressUI) { def listener = parent.listener - def stageTable: StageTable = parent.stageTable - - def poolTable: PoolTable = parent.poolTable - def render(request: HttpServletRequest): Seq[Node] = { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages val activeStages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]).toSeq - val stageToPool = listener.stageToPool val poolDetailPoolSource = new PoolDetailSource(parent.sc, poolName) - poolTable.setPoolSource(poolDetailPoolSource) + val poolTable = new PoolTable(poolDetailPoolSource, listener) - stageTable.setStagePoolInfo(parent.stagePoolInfo) + val activeStagesTable = new StageTable(activeStages, parent) - val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) + val content =

    Pool

    ++ poolTable.toNodeSeq() ++ +

    Active Stages : {activeStages.size}

    ++ activeStagesTable.toNodeSeq() - val content =

    Pool

    ++ poolTable.toNodeSeq ++ -

    Active Stages : {activeStages.size}

    ++ activeStageNodeSeq - headerSparkPage(content, parent.sc, "Spark Pool Details", Jobs) } } diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala index bb8be4b26e..8788ed8bc1 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -27,7 +27,7 @@ private[spark] trait PoolSource { /* * Pool source for FIFO scheduler algorithm on Index page */ -private[spark] class FIFOSource() extends PoolSource{ +private[spark] class FIFOSource() extends PoolSource { def getPools: Seq[Schedulable] = { Seq[Schedulable]() } @@ -36,16 +36,16 @@ private[spark] class FIFOSource() extends PoolSource{ /* * Pool source for Fair scheduler algorithm on Index page */ -private[spark] class FairSource(sc: SparkContext) extends PoolSource{ +private[spark] class FairSource(sc: SparkContext) extends PoolSource { def getPools: Seq[Schedulable] = { - sc.getPoolsInfo.toSeq + sc.getPools.toSeq } } /* * specific pool info for pool detail page */ -private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource{ +private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource { def getPools: Seq[Schedulable] = { val pools = HashSet[Schedulable]() pools += sc.getPoolNameToPool(poolName) @@ -54,21 +54,18 @@ private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extend } /** Table showing list of pools */ -private[spark] class PoolTable(listener: JobProgressListener) { +private[spark] class PoolTable(poolSource: PoolSource, listener: JobProgressListener) { - var poolSource: PoolSource = null var poolToActiveStages: HashMap[String, HashSet[Stage]] = listener.poolToActiveStages - def toNodeSeq: Seq[Node] = { + def toNodeSeq(): Seq[Node] = { poolTable(poolRow, poolSource.getPools) } - def setPoolSource(poolSource: PoolSource) { - this.poolSource = poolSource - } - - //pool tables - def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[Stage]]) => Seq[Node], rows: Seq[Schedulable]): Seq[Node] = { + // pool tables + def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[Stage]]) => Seq[Node], + rows: Seq[Schedulable] + ): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 83e566c55b..82fb0bd5cc 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -44,18 +44,18 @@ private[spark] class FairStagePoolInfo(listener: JobProgressListener) extends St } /** Page showing list of all ongoing and recently finished stages */ -private[spark] class StageTable(val dateFmt: SimpleDateFormat, val formatDuration: Long => String, val listener: JobProgressListener) { +private[spark] class StageTable( + val stages: Seq[Stage], + val parent: JobProgressUI) { - var stagePoolInfo: StagePoolInfo = null + val listener = parent.listener + val dateFmt = parent.dateFmt + var stagePoolInfo = parent.stagePoolInfo - def toNodeSeq(stages: Seq[Stage]): Seq[Node] = { + def toNodeSeq(): Seq[Node] = { stageTable(stageRow, stages) } - def setStagePoolInfo(stagePoolInfo: StagePoolInfo) { - this.stagePoolInfo = stagePoolInfo - } - /** Special table which merges two header cells. */ def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = {
    Pool Name
    @@ -77,7 +77,7 @@ private[spark] class StageTable(val dateFmt: SimpleDateFormat, val formatDuratio def getElapsedTime(submitted: Option[Long], completed: Long): String = { submitted match { - case Some(t) => formatDuration(completed - t) + case Some(t) => parent.formatDuration(completed - t) case _ => "Unknown" } } From 5406013997d6b9e9e3f43c09cd3f53ec7b815efb Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 30 Jul 2013 11:41:38 +0800 Subject: [PATCH 352/419] refactor codes less than 100 character per line --- core/src/main/scala/spark/SparkContext.scala | 7 ++++--- .../main/scala/spark/scheduler/DAGScheduler.scala | 3 ++- .../scala/spark/scheduler/SparkListener.scala | 15 ++++++++------- .../scheduler/cluster/ClusterScheduler.scala | 5 +++-- .../scheduler/cluster/ClusterTaskSetManager.scala | 3 ++- .../scheduler/cluster/SchedulableBuilder.scala | 15 ++++++++++----- .../spark/scheduler/cluster/SchedulingMode.scala | 3 ++- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 9 ++++++--- .../scala/spark/ui/jobs/JobProgressListener.scala | 3 ++- 9 files changed, 39 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 375636071d..7747160290 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -44,9 +44,10 @@ import org.apache.mesos.MesosNativeLibrary import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} -import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable, -SchedulingMode} +import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, + SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, + ClusterScheduler, Schedulable, SchedulingMode} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{StorageStatus, StorageUtils, RDDInfo} diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c865743e37..5fda78e152 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -473,7 +473,8 @@ class DAGScheduler( } if (tasks.size > 0) { val properties = idToActiveJob(stage.priority).properties - sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size, properties))) + sparkListeners.foreach(_.onStageSubmitted( + SparkListenerStageSubmitted(stage, tasks.size, properties))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 07372ee786..49f7c85c29 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -8,17 +8,18 @@ import spark.executor.TaskMetrics sealed trait SparkListenerEvents -case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties) extends SparkListenerEvents +case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties) + extends SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvents -case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) +case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) extends SparkListenerEvents -case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) +case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) extends SparkListenerEvents trait SparkListener { @@ -26,12 +27,12 @@ trait SparkListener { * Called when a stage is completed, with information on the completed stage */ def onStageCompleted(stageCompleted: StageCompleted) { } - + /** * Called when a stage is submitted */ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } - + /** * Called when a task ends */ @@ -41,12 +42,12 @@ trait SparkListener { * Called when a job starts */ def onJobStart(jobStart: SparkListenerJobStart) { } - + /** * Called when a job ends */ def onJobEnd(jobEnd: SparkListenerJobEnd) { } - + } /** diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 74b3e43d2b..20680bbf87 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -9,10 +9,10 @@ import scala.collection.mutable.HashSet import spark._ import spark.TaskState.TaskState import spark.scheduler._ +import spark.scheduler.cluster.SchedulingMode.SchedulingMode import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong import java.util.{TimerTask, Timer} -import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call @@ -257,7 +257,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() for (manager <- sortedTaskSetQueue) { - logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) + logInfo("parentName:%s, name:%s, runningTasks:%s".format( + manager.parent.name, manager.name, manager.runningTasks)) } for (manager <- sortedTaskSetQueue) { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 4d11b0959a..5e2351bafd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -644,7 +644,8 @@ private[spark] class ClusterTaskSetManager( } } - // TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + // TODO: for now we just find Pool not TaskSetManager + // we can extend this function in future if needed override def getSchedulableByName(name: String): Schedulable = { return null } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index a2fa80aa36..db51b48494 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -24,7 +24,8 @@ private[spark] trait SchedulableBuilder { def addTaskSetManager(manager: Schedulable, properties: Properties) } -private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { +private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) + extends SchedulableBuilder with Logging { override def buildPools() { // nothing @@ -35,7 +36,8 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends Schedula } } -private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { +private[spark] class FairSchedulableBuilder(val rootPool: Pool) + extends SchedulableBuilder with Logging { val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" @@ -88,7 +90,8 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula // finally create "default" pool if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) { - val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) + val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, + DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) rootPool.addSchedulable(pool) logInfo("Create default pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) @@ -102,8 +105,10 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula poolName = properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) parentPool = rootPool.getSchedulableByName(poolName) if (parentPool == null) { - // we will create a new pool that user has configured in app instead of being defined in xml file - parentPool = new Pool(poolName,DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) + // we will create a new pool that user has configured in app + // instead of being defined in xml file + parentPool = new Pool(poolName, DEFAULT_SCHEDULING_MODE, + DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) rootPool.addSchedulable(parentPool) logInfo("Create pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( poolName, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index a7f0f6f393..cd0642772d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,7 +1,8 @@ package spark.scheduler.cluster /** - * "FAIR" and "FIFO" determines which policy is used to order tasks amongst a Schedulable's sub-queues + * "FAIR" and "FIFO" determines which policy is used + * to order tasks amongst a Schedulable's sub-queues * "NONE" is used when the a Schedulable has no sub-queues. */ object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE") { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index abef683791..04651e9c60 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -40,9 +40,12 @@ private[spark] class IndexPage(parent: JobProgressUI) { ++

    Pools

    ++ poolTable.toNodeSeq ++ -

    Active Stages : {activeStages.size}

    ++ activeStagesTable.toNodeSeq++ -

    Completed Stages : {completedStages.size}

    ++ completedStagesTable.toNodeSeq++ -

    Failed Stages : {failedStages.size}

    ++ failedStagesTable.toNodeSeq +

    Active Stages : {activeStages.size}

    ++ + activeStagesTable.toNodeSeq++ +

    Completed Stages : {completedStages.size}

    ++ + completedStagesTable.toNodeSeq++ +

    Failed Stages : {failedStages.size}

    ++ + failedStagesTable.toNodeSeq headerSparkPage(content, parent.sc, "Spark Stages/Pools", Jobs) } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala index d4767bea22..da767b3c0a 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala @@ -53,7 +53,8 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList activeStages += stage var poolName = DEFAULT_POOL_NAME if (stageSubmitted.properties != null) { - poolName = stageSubmitted.properties.getProperty("spark.scheduler.cluster.fair.pool", DEFAULT_POOL_NAME) + poolName = stageSubmitted.properties.getProperty("spark.scheduler.cluster.fair.pool", + DEFAULT_POOL_NAME) } stageToPool(stage) = poolName val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) From b95732632b5b06d494ebd9e539af136ab3b8490e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 28 Jul 2013 19:24:23 -0700 Subject: [PATCH 353/419] Do not inherit master's PYTHONPATH on workers. This fixes SPARK-832, an issue where PySpark would not work when the master and workers used different SPARK_HOME paths. This change may potentially break code that relied on the master's PYTHONPATH being used on workers. To have custom PYTHONPATH additions used on the workers, users should set a custom PYTHONPATH in spark-env.sh rather than setting it in the shell. --- .../main/scala/spark/api/python/PythonWorkerFactory.scala | 2 ++ python/pyspark/rdd.py | 5 ++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 078ad45ce8..84673470db 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -67,6 +67,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) val workerEnv = pb.environment() workerEnv.putAll(envVars) + val pythonPath = sparkHome + "/python/:" + workerEnv.get("PYTHONPATH") + workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() // Redirect the stderr to ours diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 8734cacb0b..51c2cb9806 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -756,9 +756,8 @@ class PipelinedRDD(RDD): self.ctx._gateway._gateway_client) self.ctx._pickled_broadcast_vars.clear() class_manifest = self._prev_jrdd.classManifest() - env = copy.copy(self.ctx.environment) - env['PYTHONPATH'] = os.environ.get("PYTHONPATH", "") - env = MapConverter().convert(env, self.ctx._gateway._gateway_client) + env = MapConverter().convert(self.ctx.environment, + self.ctx._gateway._gateway_client) python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(), pipe_command, env, self.preservesPartitioning, self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator, class_manifest) From 49be084ed3cca4778ff0602c0bee962a3515338e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 28 Jul 2013 21:21:19 -0700 Subject: [PATCH 354/419] Use File.pathSeparator instead of hardcoding ':'. --- .../src/main/scala/spark/api/python/PythonWorkerFactory.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 84673470db..14f8320678 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -17,7 +17,7 @@ package spark.api.python -import java.io.{DataInputStream, IOException} +import java.io.{File, DataInputStream, IOException} import java.net.{Socket, SocketException, InetAddress} import scala.collection.JavaConversions._ @@ -67,7 +67,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/:" + workerEnv.get("PYTHONPATH") + val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() From e4387ddf5d1a46dfedece73feff4de6a30f9a220 Mon Sep 17 00:00:00 2001 From: Ameet Talwalkar Date: Mon, 29 Jul 2013 22:21:50 -0700 Subject: [PATCH 355/419] made SimpleUpdater consistent with other updaters --- mllib/src/main/scala/spark/mllib/optimization/Updater.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index e916a92c33..bf506d2f24 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -41,7 +41,8 @@ abstract class Updater extends Serializable { class SimpleUpdater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { - val normGradient = gradient.mul(stepSize / math.sqrt(iter)) + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) (weightsOld.sub(normGradient), 0) } } From f6f46455eb4f1199eb9a464b1a0b45d9926f7ff8 Mon Sep 17 00:00:00 2001 From: Benjamin Hindman Date: Tue, 23 Jul 2013 09:33:13 -0400 Subject: [PATCH 356/419] Added property 'spark.executor.uri' for launching on Mesos without requiring Spark to be installed. Using 'make_distribution.sh' a user can put a Spark distribution at a URI supported by Mesos (e.g., 'hdfs://...') and then set that when launching their job. Also added SPARK_EXECUTOR_URI for the REPL. --- .../mesos/CoarseMesosSchedulerBackend.scala | 27 ++++++++++++++----- .../mesos/MesosSchedulerBackend.scala | 17 ++++++++---- make-distribution.sh | 1 + project/SparkBuild.scala | 2 +- .../main/scala/spark/repl/SparkILoop.scala | 2 ++ 5 files changed, 36 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 7bc6040544..f75244a9ac 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -110,12 +110,6 @@ private[spark] class CoarseMesosSchedulerBackend( } def createCommand(offer: Offer, numCores: Int): CommandInfo = { - val runScript = new File(sparkHome, "run").getCanonicalPath - val driverUrl = "akka://spark@%s:%s/user/%s".format( - System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), - StandaloneSchedulerBackend.ACTOR_NAME) - val command = "\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores) val environment = Environment.newBuilder() sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() @@ -123,7 +117,26 @@ private[spark] class CoarseMesosSchedulerBackend( .setValue(value) .build()) } - return CommandInfo.newBuilder().setValue(command).setEnvironment(environment).build() + val command = CommandInfo.newBuilder() + .setEnvironment(environment) + val driverUrl = "akka://spark@%s:%s/user/%s".format( + System.getProperty("spark.driver.host"), + System.getProperty("spark.driver.port"), + StandaloneSchedulerBackend.ACTOR_NAME) + val uri = System.getProperty("spark.executor.uri") + if (uri == null) { + val runScript = new File(sparkHome, "run").getCanonicalPath + command.setValue("\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format( + runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + } else { + // Grab everything to the first '.'. We'll use that and '*' to + // glob the directory "correctly". + val basename = new File(uri).getName().split('.')(0) + command.setValue("cd %s*; ./run spark.executor.StandaloneExecutorBackend %s %s %s %d".format( + basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) + } + return command.build() } override def offerRescinded(d: SchedulerDriver, o: OfferID) {} diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 75b8268b55..51b780ac72 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -89,7 +89,6 @@ private[spark] class MesosSchedulerBackend( val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) - val execScript = new File(sparkHome, "spark-executor").getCanonicalPath val environment = Environment.newBuilder() sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() @@ -97,15 +96,23 @@ private[spark] class MesosSchedulerBackend( .setValue(value) .build()) } + val command = CommandInfo.newBuilder() + .setEnvironment(environment) + val uri = System.getProperty("spark.executor.uri") + if (uri == null) { + command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath) + } else { + // Grab everything to the first '.'. We'll use that and '*' to + // glob the directory "correctly". + val basename = new File(uri).getName().split('.')(0) + command.setValue("cd %s*; ./spark-executor".format(basename)) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) + } val memory = Resource.newBuilder() .setName("mem") .setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build()) .build() - val command = CommandInfo.newBuilder() - .setValue(execScript) - .setEnvironment(environment) - .build() ExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) diff --git a/make-distribution.sh b/make-distribution.sh index 4374e0e8c4..0a8941c1f8 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -66,6 +66,7 @@ cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/conf" "$DISTDIR" cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" +cp "$FWDIR/spark-executor" "$DISTDIR" if [ "$1" == "tgz" ]; then diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9920e00a67..a2ea9c9694 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -176,7 +176,7 @@ object SparkBuild extends Build { "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", - "org.apache.mesos" % "mesos" % "0.9.0-incubating", + "org.apache.mesos" % "mesos" % "0.12.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 59f9d05683..0bfe7bb743 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -831,6 +831,8 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: var sparkContext: SparkContext = null def createSparkContext(): SparkContext = { + val uri = System.getenv("SPARK_EXECUTOR_URI") + if (uri != null) System.setProperty("spark.executor.uri", uri) val master = this.master match { case Some(m) => m case None => { From f1cab31b73a7f5464f03b9b24b1338d5bb1496da Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 30 Jul 2013 11:06:47 -0700 Subject: [PATCH 357/419] Removed intermediate set for activeTasks, removed progress bar margin --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 3 ++- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index d6f83157d3..bc1a8ae31a 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -93,7 +93,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, HashSet[TaskInfo]()).size.toString + val activeTasks = listener.executorToTasksActive.get(a.toString).map(l => l.size). + getOrElse(0).toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index e2cf84346a..157418fa79 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -111,7 +111,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) val startWidth = "width: %s%%".format((started.toDouble/total)*100) -
    +
    From 218d7c4ed8fef3fae0da0d3adc1aed65e12c5627 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 30 Jul 2013 12:39:17 -0700 Subject: [PATCH 358/419] Fixed style, lowered height of progress bars --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 157418fa79..a843b5ea2f 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -111,7 +111,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) val startWidth = "width: %s%%".format((started.toDouble/total)*100) -
    +
    From 26144c400f176d74714adc0a7b4168031d868413 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 30 Jul 2013 12:40:41 -0700 Subject: [PATCH 359/419] Fixed wrap style --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index bc1a8ae31a..4be2bfa413 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -93,8 +93,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.get(a.toString).map(l => l.size). - getOrElse(0).toString + val activeTasks = listener.executorToTasksActive.get(a.toString).map(l => l.size) + .getOrElse(0).toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString From e35966ae9a0db3cfed9c46923d939fcc3d4eec4e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 13:28:31 -0700 Subject: [PATCH 360/419] Renamed Classification.scala to ClassificationModel.scala and Regression.scala to RegressionModel.scala --- .../{Classification.scala => ClassificationModel.scala} | 0 .../mllib/regression/{Regression.scala => RegressionModel.scala} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename mllib/src/main/scala/spark/mllib/classification/{Classification.scala => ClassificationModel.scala} (100%) rename mllib/src/main/scala/spark/mllib/regression/{Regression.scala => RegressionModel.scala} (100%) diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/classification/Classification.scala rename to mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/regression/Regression.scala rename to mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala From 47011e685449edfe9f91d8f937a5d23d3b359baf Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 13:58:23 -0700 Subject: [PATCH 361/419] Use a tigher bound in logistic regression unit test's prediction validation. --- .../mllib/classification/LogisticRegressionSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index d3fe58a382..8664263935 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -21,11 +21,12 @@ import scala.util.Random import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers import spark.SparkContext -class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { +class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll with ShouldMatchers { val sc = new SparkContext("local", "test") override def afterAll() { @@ -64,8 +65,8 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val numOffPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => (prediction != expected) }.size - // At least 80% of the predictions should be on. - assert(numOffPredictions < input.length / 5) + // At least 83% of the predictions should be on. + ((input.length - numOffPredictions).toDouble / input.length) should be > 0.83 } // Test if we can correctly learn A, B where Y = logistic(A + B*X) From 366f7735ebe1004acf113df257950d287c50471a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 13:59:32 -0700 Subject: [PATCH 362/419] Minor style cleanup of mllib. --- .../classification/LogisticRegression.scala | 29 ++++++++++--------- .../spark/mllib/classification/SVM.scala | 15 +++++----- .../spark/mllib/optimization/Updater.scala | 10 +++---- .../spark/mllib/recommendation/ALS.scala | 5 ++-- .../scala/spark/mllib/regression/Lasso.scala | 15 +++++----- 5 files changed, 39 insertions(+), 35 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index bf3b05dedb..203aa8fdd4 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -135,8 +135,8 @@ class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBa object LogisticRegressionLocalRandomSGD { /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in * gradient descent are initialized using the initial weights provided. * @@ -155,12 +155,13 @@ object LogisticRegressionLocalRandomSGD { initialWeights: Array[Double]) : LogisticRegressionModel = { - new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) + new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train( + input, initialWeights) } /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. * * @param input RDD of (label, array of features) pairs. @@ -180,9 +181,9 @@ object LogisticRegressionLocalRandomSGD { } /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to update - * the gradient in each iteration. + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using the specified step size. We use the entire data + * set to update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. @@ -200,9 +201,9 @@ object LogisticRegressionLocalRandomSGD { } /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update - * the gradient in each iteration. + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using a step size of 1.0. We use the entire data set + * to update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. @@ -218,12 +219,14 @@ object LogisticRegressionLocalRandomSGD { def main(args: Array[String]) { if (args.length != 5) { - println("Usage: LogisticRegression ") + println("Usage: LogisticRegression " + + " ") System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)).map(yx => (yx._1.toInt, yx._2)) - val model = LogisticRegressionLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LogisticRegressionLocalRandomSGD.train( + data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 15b689e7e0..3a6a12814a 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -53,8 +53,8 @@ class SVMModel( -class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, - var numIters: Int) +class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, + var miniBatchFraction: Double, var numIters: Int) extends Logging { /** @@ -163,7 +163,8 @@ object SVMLocalRandomSGD { initialWeights: Array[Double]) : SVMModel = { - new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train( + input, initialWeights) } /** @@ -190,8 +191,8 @@ object SVMLocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to update - * the gradient in each iteration. + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. @@ -211,8 +212,8 @@ object SVMLocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update - * the gradient in each iteration. + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index bf506d2f24..3ebc1409b6 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -25,7 +25,7 @@ abstract class Updater extends Serializable { * Compute an updated value for weights given the gradient, stepSize and iteration number. * Also returns the regularization value computed using the *updated* weights. * - * @param weightsOlds - Column matrix of size nx1 where n is the number of features. + * @param weightsOld - Column matrix of size nx1 where n is the number of features. * @param gradient - Column matrix of size nx1 where n is the number of features. * @param stepSize - step size across iterations * @param iter - Iteration number @@ -34,8 +34,8 @@ abstract class Updater extends Serializable { * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, * and the second element is the regularization value computed using updated weights. */ - def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): - (DoubleMatrix, Double) + def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, + regParam: Double): (DoubleMatrix, Double) } class SimpleUpdater extends Updater { @@ -64,10 +64,10 @@ class L1Updater extends Updater { val newWeights = weightsOld.sub(normGradient) // Soft thresholding val shrinkageVal = regParam * thisIterStepSize - (0 until newWeights.length).foreach(i => { + (0 until newWeights.length).foreach { i => val wi = newWeights.get(i) newWeights.put(i, signum(wi) * max(0.0, abs(wi) - shrinkageVal)) - }) + } (newWeights, newWeights.norm1 * regParam) } } diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 7da96397a6..7281b2fcb9 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -35,8 +35,7 @@ import org.jblas.{DoubleMatrix, SimpleBlas, Solve} * of the elements within this block, and the list of destination blocks that each user or * product will need to send its feature vector to. */ -private[recommendation] case class OutLinkBlock( - elementIds: Array[Int], shouldSend: Array[BitSet]) +private[recommendation] case class OutLinkBlock(elementIds: Array[Int], shouldSend: Array[BitSet]) /** @@ -105,7 +104,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l } /** - * Run ALS with the configured parmeters on an input RDD of (user, product, rating) triples. + * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. * Returns a MatrixFactorizationModel with feature vectors for each user and product. */ def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index 1952658bb2..e8b1ed8a48 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -53,8 +53,8 @@ class LassoModel( } -class LassoLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, - var numIters: Int) +class LassoLocalRandomSGD private (var stepSize: Double, var regParam: Double, + var miniBatchFraction: Double, var numIters: Int) extends Logging { /** @@ -163,7 +163,8 @@ object LassoLocalRandomSGD { initialWeights: Array[Double]) : LassoModel = { - new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train( + input, initialWeights) } /** @@ -190,8 +191,8 @@ object LassoLocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to update - * the gradient in each iteration. + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. @@ -211,8 +212,8 @@ object LassoLocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update - * the gradient in each iteration. + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. From 48851d4dd90184fdeee836caef5ad77b0dc643be Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 30 Jul 2013 14:03:15 -0700 Subject: [PATCH 363/419] Add bagel, mllib to SBT assembly. Also add jblas dependency to mllib pom.xml --- mllib/pom.xml | 165 +++++++++++++++++++++++++++++++++++++++ pom.xml | 1 + project/SparkBuild.scala | 6 +- 3 files changed, 170 insertions(+), 2 deletions(-) create mode 100644 mllib/pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml new file mode 100644 index 0000000000..f3928cc73d --- /dev/null +++ b/mllib/pom.xml @@ -0,0 +1,165 @@ + + + + + 4.0.0 + + org.spark-project + spark-parent + 0.8.0-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-mllib + jar + Spark Project ML Library + http://spark-project.org/ + + + + org.eclipse.jetty + jetty-server + + + org.jblas + jblas + 1.2.3 + + + + org.scalatest + scalatest_${scala.version} + test + + + org.scalacheck + scalacheck_${scala.version} + test + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + + + + hadoop1 + + + org.spark-project + spark-core + ${project.version} + hadoop1 + + + org.apache.hadoop + hadoop-core + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop1 + + + + + + + hadoop2 + + + org.spark-project + spark-core + ${project.version} + hadoop2 + + + org.apache.hadoop + hadoop-core + provided + + + org.apache.hadoop + hadoop-client + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2 + + + + + + + hadoop2-yarn + + + org.spark-project + spark-core + ${project.version} + hadoop2-yarn + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2-yarn + + + + + + + diff --git a/pom.xml b/pom.xml index 44729bd422..03b36f2ee8 100644 --- a/pom.xml +++ b/pom.xml @@ -58,6 +58,7 @@ core bagel examples + mllib tools streaming repl diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9920e00a67..0fdd291aac 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -242,14 +242,16 @@ object SparkBuild extends Build { name := "spark-tools" ) - def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") + def bagelSettings = sharedSettings ++ Seq( + name := "spark-bagel" + ) ++ assemblySettings ++ extraAssemblySettings def mllibSettings = sharedSettings ++ Seq( name := "spark-mllib", libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3" ) - ) + ) ++ assemblySettings ++ extraAssemblySettings def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", From 7bdafa918ac5850197dacadd3ea56ef544a6a9f8 Mon Sep 17 00:00:00 2001 From: Benjamin Hindman Date: Tue, 30 Jul 2013 17:01:00 -0700 Subject: [PATCH 364/419] Format cleanup. --- repl/src/main/scala/spark/repl/SparkILoop.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 0bfe7bb743..0cecbd71ad 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -832,7 +832,9 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def createSparkContext(): SparkContext = { val uri = System.getenv("SPARK_EXECUTOR_URI") - if (uri != null) System.setProperty("spark.executor.uri", uri) + if (uri != null) { + System.setProperty("spark.executor.uri", uri) + } val master = this.master match { case Some(m) => m case None => { From ad7e9d0d64277f616f90f2ca8bf8a5844641883a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:11:54 -0700 Subject: [PATCH 365/419] CompressionCodec cleanup. Moved it to spark.io package. --- .../scala/spark/io/CompressionCodec.scala | 82 +++++++++++++++++++ .../scala/spark/storage/BlockManager.scala | 32 ++------ .../spark/storage/CompressionCodec.scala | 13 --- .../spark/storage/LZFCompressionCodec.scala | 16 ---- .../storage/SnappyCompressionCodec.scala | 18 ---- .../spark/io/CompressionCodecSuite.scala | 0 6 files changed, 91 insertions(+), 70 deletions(-) create mode 100644 core/src/main/scala/spark/io/CompressionCodec.scala delete mode 100644 core/src/main/scala/spark/storage/CompressionCodec.scala delete mode 100644 core/src/main/scala/spark/storage/LZFCompressionCodec.scala delete mode 100644 core/src/main/scala/spark/storage/SnappyCompressionCodec.scala create mode 100644 core/src/test/scala/spark/io/CompressionCodecSuite.scala diff --git a/core/src/main/scala/spark/io/CompressionCodec.scala b/core/src/main/scala/spark/io/CompressionCodec.scala new file mode 100644 index 0000000000..2ba104a737 --- /dev/null +++ b/core/src/main/scala/spark/io/CompressionCodec.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.io + +import java.io.{InputStream, OutputStream} + +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} + +import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} + + +/** + * CompressionCodec allows the customization of choosing different compression implementations + * to be used in block storage. + */ +trait CompressionCodec { + + def compressionOutputStream(s: OutputStream): OutputStream + + def compressionInputStream(s: InputStream): InputStream +} + + +private[spark] object CompressionCodec { + + def createCodec(): CompressionCodec = { + // Set the default codec to Snappy since the LZF implementation initializes a pretty large + // buffer for every stream, which results in a lot of memory overhead when the number of + // shuffle reduce buckets are large. + createCodec(classOf[SnappyCompressionCodec].getName) + } + + def createCodec(codecName: String): CompressionCodec = { + Class.forName( + System.getProperty("spark.io.compression.codec", codecName), + true, + Thread.currentThread.getContextClassLoader).newInstance().asInstanceOf[CompressionCodec] + } +} + + +/** + * LZF implementation of [[spark.io.CompressionCodec]]. + */ +class LZFCompressionCodec extends CompressionCodec { + + override def compressionOutputStream(s: OutputStream): OutputStream = { + new LZFOutputStream(s).setFinishBlockOnFlush(true) + } + + override def compressionInputStream(s: InputStream): InputStream = new LZFInputStream(s) +} + + +/** + * Snappy implementation of [[spark.io.CompressionCodec]]. + * Block size can be configured by spark.io.compression.snappy.block.size. + */ +class SnappyCompressionCodec extends CompressionCodec { + + override def compressionOutputStream(s: OutputStream): OutputStream = { + val blockSize = System.getProperty("spark.io.snappy.block.size", "32768").toInt + new SnappyOutputStream(s, blockSize) + } + + override def compressionInputStream(s: InputStream): InputStream = new SnappyInputStream(s) +} diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 4228c902f8..9ed4c01218 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -27,11 +27,10 @@ import akka.dispatch.{Await, Future} import akka.util.Duration import akka.util.duration._ -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} - import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import spark.{Logging, SparkEnv, SparkException, Utils} +import spark.io.CompressionCodec import spark.network._ import spark.serializer.Serializer import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} @@ -158,7 +157,12 @@ private[spark] class BlockManager( val metadataCleaner = new MetadataCleaner("BlockManager", this.dropOldBlocks) initialize() - var compressionCodec: CompressionCodec = null + // The compression codec to use. Note that the "lazy" val is necessary because we want to delay + // the initialization of the compression codec until it is first used. The reason is that a Spark + // program could be using a user-defined codec in a third party jar, which is loaded in + // Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been + // loaded yet. + private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec() /** * Construct a BlockManager with a memory limit set based on system properties. @@ -921,32 +925,14 @@ private[spark] class BlockManager( * Wrap an output stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: OutputStream): OutputStream = { - if (compressionCodec == null) { - compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", - "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) - .newInstance().asInstanceOf[CompressionCodec] - } - - if (shouldCompress(blockId)) { - //(new LZFOutputStream(s)).setFinishBlockOnFlush(true) - compressionCodec.compressionOutputStream(s) - } else { - s - } + if (shouldCompress(blockId)) compressionCodec.compressionOutputStream(s) else s } /** * Wrap an input stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: InputStream): InputStream = { - if (compressionCodec == null) { - compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", - "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) - .newInstance().asInstanceOf[CompressionCodec] - } - - if (shouldCompress(blockId)) /*new LZFInputStream(s) */ - compressionCodec.compressionInputStream(s) else s + if (shouldCompress(blockId)) compressionCodec.compressionInputStream(s) else s } def dataSerialize( diff --git a/core/src/main/scala/spark/storage/CompressionCodec.scala b/core/src/main/scala/spark/storage/CompressionCodec.scala deleted file mode 100644 index cd80de33f6..0000000000 --- a/core/src/main/scala/spark/storage/CompressionCodec.scala +++ /dev/null @@ -1,13 +0,0 @@ -package spark.storage - -import java.io.{InputStream, OutputStream} - - -/** - * CompressionCodec allows the customization of the compression codec - */ -trait CompressionCodec { - def compressionOutputStream(s: OutputStream): OutputStream - - def compressionInputStream(s: InputStream): InputStream -} diff --git a/core/src/main/scala/spark/storage/LZFCompressionCodec.scala b/core/src/main/scala/spark/storage/LZFCompressionCodec.scala deleted file mode 100644 index 3328b949ef..0000000000 --- a/core/src/main/scala/spark/storage/LZFCompressionCodec.scala +++ /dev/null @@ -1,16 +0,0 @@ -package spark.storage - -import java.io.{InputStream, OutputStream} - -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} - -/** - * LZF implementation of [[spark.storage.CompressionCodec]] - */ -class LZFCompressionCodec extends CompressionCodec { - def compressionOutputStream(s: OutputStream): OutputStream = - (new LZFOutputStream(s)).setFinishBlockOnFlush(true) - - def compressionInputStream(s: InputStream): InputStream = - new LZFInputStream(s) -} diff --git a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala deleted file mode 100644 index 62b00ef3f6..0000000000 --- a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala +++ /dev/null @@ -1,18 +0,0 @@ -package spark.storage - -import java.io.{InputStream, OutputStream} - -import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} - -/** - * Snappy implementation of [[spark.storage.CompressionCodec]] - * block size can be configured by spark.snappy.block.size - */ -class SnappyCompressionCodec extends CompressionCodec { - def compressionOutputStream(s: OutputStream): OutputStream = - new SnappyOutputStream(s, - System.getProperty("spark.snappy.block.size", "32768").toInt) - - def compressionInputStream(s: InputStream): InputStream = - new SnappyInputStream(s) -} diff --git a/core/src/test/scala/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/spark/io/CompressionCodecSuite.scala new file mode 100644 index 0000000000..e69de29bb2 From 5227043f84e29bdca9a3be95d03886b0acea4ac6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:12:16 -0700 Subject: [PATCH 366/419] Documentation update for compression codec. --- docs/configuration.md | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 5c06897cae..0bcd73ca99 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -35,7 +35,7 @@ for these variables. * `SPARK_JAVA_OPTS`, to add JVM options. This includes any system properties that you'd like to pass with `-D`. * `SPARK_CLASSPATH`, to add elements to Spark's classpath. * `SPARK_LIBRARY_PATH`, to add search directories for native libraries. -* `SPARK_MEM`, to set the amount of memory used per node. This should be in the same format as the +* `SPARK_MEM`, to set the amount of memory used per node. This should be in the same format as the JVM's -Xmx option, e.g. `300m` or `1g`. Note that this option will soon be deprecated in favor of the `spark.executor.memory` system property, so we recommend using that in new code. @@ -77,7 +77,7 @@ there are at least five properties that you will commonly want to control: Class to use for serializing objects that will be sent over the network or need to be cached in serialized form. The default of Java serialization works with any Serializable Java object but is quite slow, so we recommend using spark.KryoSerializer - and configuring Kryo serialization when speed is necessary. Can be any subclass of + and configuring Kryo serialization when speed is necessary. Can be any subclass of spark.Serializer). @@ -86,7 +86,7 @@ there are at least five properties that you will commonly want to control:
    @@ -180,6 +180,21 @@ Apart from these, the following properties are also available, and may be useful Can save substantial space at the cost of some extra CPU time. + + + + + + + + + + From 56774b176eb7e7a556bb23d9c524621e156c5633 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:12:33 -0700 Subject: [PATCH 367/419] Added unit test for compression codecs. --- .../spark/io/CompressionCodecSuite.scala | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) diff --git a/core/src/test/scala/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/spark/io/CompressionCodecSuite.scala index e69de29bb2..1b5daf4c97 100644 --- a/core/src/test/scala/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/spark/io/CompressionCodecSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.io + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +import org.scalatest.FunSuite + + +class CompressionCodecSuite extends FunSuite { + + def testCodec(codec: CompressionCodec) { + // Write 1000 integers to the output stream, compressed. + val outputStream = new ByteArrayOutputStream() + val out = codec.compressionOutputStream(outputStream) + for (i <- 1 until 1000) { + out.write(i % 256) + } + out.close() + + // Read the 1000 integers back. + val inputStream = new ByteArrayInputStream(outputStream.toByteArray) + val in = codec.compressionInputStream(inputStream) + for (i <- 1 until 1000) { + assert(in.read() === i % 256) + } + in.close() + } + + test("default compression codec") { + val codec = CompressionCodec.createCodec() + assert(codec.getClass === classOf[SnappyCompressionCodec]) + testCodec(codec) + } + + test("lzf compression codec") { + val codec = CompressionCodec.createCodec(classOf[LZFCompressionCodec].getName) + assert(codec.getClass === classOf[LZFCompressionCodec]) + testCodec(codec) + } + + test("snappy compression codec") { + val codec = CompressionCodec.createCodec(classOf[SnappyCompressionCodec].getName) + assert(codec.getClass === classOf[SnappyCompressionCodec]) + testCodec(codec) + } +} From 3b1ced83fbd72703965aabf7f8dc52417e1cf166 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:24:45 -0700 Subject: [PATCH 368/419] Exclude older version of Snappy in streaming and examples. --- project/SparkBuild.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index be3ef1f148..07d7b76901 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -151,6 +151,7 @@ object SparkBuild extends Build { val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") val excludeNetty = ExclusionRule(organization = "org.jboss.netty") val excludeAsm = ExclusionRule(organization = "asm") + val excludeSnappy = ExclusionRule(organization = "org.xerial.snappy") def coreSettings = sharedSettings ++ Seq( name := "spark-core", @@ -236,6 +237,7 @@ object SparkBuild extends Build { exclude("jline","jline") exclude("log4j","log4j") exclude("org.apache.cassandra.deps", "avro") + excludeAll(excludeSnappy) ) ) @@ -258,7 +260,7 @@ object SparkBuild extends Build { "Akka Repository" at "http://repo.akka.io/releases/" ), libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), + "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty) From 311aae76a2177629a6afb75a36563b36e3aa7b66 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:25:42 -0700 Subject: [PATCH 369/419] Added Snappy dependency to Maven build files. --- core/pom.xml | 4 ++++ pom.xml | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index f0c936c86a..ba0071f582 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -48,6 +48,10 @@ com.ning compress-lzf + + org.xerial.snappy + snappy-java + org.ow2.asm asm diff --git a/pom.xml b/pom.xml index 44729bd422..f3bca6c40b 100644 --- a/pom.xml +++ b/pom.xml @@ -182,6 +182,11 @@ compress-lzf 0.8.4 + + org.xerial.snappy + snappy-java + 1.0.5 + org.ow2.asm asm From dae12fef9ed473a9dcdfe00b04497eea21bfb96b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 17:49:31 -0700 Subject: [PATCH 370/419] Updated the configuration option for Snappy block size to be consistent with the documentation. --- core/src/main/scala/spark/io/CompressionCodec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/io/CompressionCodec.scala b/core/src/main/scala/spark/io/CompressionCodec.scala index 2ba104a737..b4d8481450 100644 --- a/core/src/main/scala/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/spark/io/CompressionCodec.scala @@ -74,7 +74,7 @@ class LZFCompressionCodec extends CompressionCodec { class SnappyCompressionCodec extends CompressionCodec { override def compressionOutputStream(s: OutputStream): OutputStream = { - val blockSize = System.getProperty("spark.io.snappy.block.size", "32768").toInt + val blockSize = System.getProperty("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } From 98024eadc3150a9a509132117875b8d0b18b1d50 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 18:28:46 -0700 Subject: [PATCH 371/419] Renamed compressionOutputStream and compressionInputStream to compressedOutputStream and compressedInputStream. --- core/src/main/scala/spark/io/CompressionCodec.scala | 12 ++++++------ core/src/main/scala/spark/storage/BlockManager.scala | 4 ++-- .../test/scala/spark/io/CompressionCodecSuite.scala | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/io/CompressionCodec.scala b/core/src/main/scala/spark/io/CompressionCodec.scala index b4d8481450..0adebecadb 100644 --- a/core/src/main/scala/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/spark/io/CompressionCodec.scala @@ -30,9 +30,9 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} */ trait CompressionCodec { - def compressionOutputStream(s: OutputStream): OutputStream + def compressedOutputStream(s: OutputStream): OutputStream - def compressionInputStream(s: InputStream): InputStream + def compressedInputStream(s: InputStream): InputStream } @@ -59,11 +59,11 @@ private[spark] object CompressionCodec { */ class LZFCompressionCodec extends CompressionCodec { - override def compressionOutputStream(s: OutputStream): OutputStream = { + override def compressedOutputStream(s: OutputStream): OutputStream = { new LZFOutputStream(s).setFinishBlockOnFlush(true) } - override def compressionInputStream(s: InputStream): InputStream = new LZFInputStream(s) + override def compressedInputStream(s: InputStream): InputStream = new LZFInputStream(s) } @@ -73,10 +73,10 @@ class LZFCompressionCodec extends CompressionCodec { */ class SnappyCompressionCodec extends CompressionCodec { - override def compressionOutputStream(s: OutputStream): OutputStream = { + override def compressedOutputStream(s: OutputStream): OutputStream = { val blockSize = System.getProperty("spark.io.compression.snappy.block.size", "32768").toInt new SnappyOutputStream(s, blockSize) } - override def compressionInputStream(s: InputStream): InputStream = new SnappyInputStream(s) + override def compressedInputStream(s: InputStream): InputStream = new SnappyInputStream(s) } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9ed4c01218..3a72474419 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -925,14 +925,14 @@ private[spark] class BlockManager( * Wrap an output stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: OutputStream): OutputStream = { - if (shouldCompress(blockId)) compressionCodec.compressionOutputStream(s) else s + if (shouldCompress(blockId)) compressionCodec.compressedOutputStream(s) else s } /** * Wrap an input stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: InputStream): InputStream = { - if (shouldCompress(blockId)) compressionCodec.compressionInputStream(s) else s + if (shouldCompress(blockId)) compressionCodec.compressedInputStream(s) else s } def dataSerialize( diff --git a/core/src/test/scala/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/spark/io/CompressionCodecSuite.scala index 1b5daf4c97..1ba82fe2b9 100644 --- a/core/src/test/scala/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/spark/io/CompressionCodecSuite.scala @@ -27,7 +27,7 @@ class CompressionCodecSuite extends FunSuite { def testCodec(codec: CompressionCodec) { // Write 1000 integers to the output stream, compressed. val outputStream = new ByteArrayOutputStream() - val out = codec.compressionOutputStream(outputStream) + val out = codec.compressedOutputStream(outputStream) for (i <- 1 until 1000) { out.write(i % 256) } @@ -35,7 +35,7 @@ class CompressionCodecSuite extends FunSuite { // Read the 1000 integers back. val inputStream = new ByteArrayInputStream(outputStream.toByteArray) - val in = codec.compressionInputStream(inputStream) + val in = codec.compressedInputStream(inputStream) for (i <- 1 until 1000) { assert(in.read() === i % 256) } From 15fd0d619d24caaf0af6ec0c323c983a8daf8faa Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 30 Jul 2013 18:31:11 -0700 Subject: [PATCH 372/419] Add mllib, bagel to repl dependencies Also don't build an assembly jar for them --- project/SparkBuild.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0fdd291aac..f36c4f2655 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -44,7 +44,7 @@ object SparkBuild extends Build { lazy val core = Project("core", file("core"), settings = coreSettings) - lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) + lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) dependsOn(bagel) dependsOn(mllib) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) @@ -244,14 +244,14 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq( name := "spark-bagel" - ) ++ assemblySettings ++ extraAssemblySettings + ) def mllibSettings = sharedSettings ++ Seq( name := "spark-mllib", libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3" ) - ) ++ assemblySettings ++ extraAssemblySettings + ) def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", From bf9318091a54cce7f8a5370f4cf273fb4846e94e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 31 Jul 2013 09:42:16 +0800 Subject: [PATCH 373/419] Add Apache license header to metrics system --- .../scala/spark/metrics/MetricsConfig.scala | 17 +++++++++++++++++ .../scala/spark/metrics/MetricsSystem.scala | 17 +++++++++++++++++ .../scala/spark/metrics/sink/ConsoleSink.scala | 17 +++++++++++++++++ .../main/scala/spark/metrics/sink/CsvSink.scala | 17 +++++++++++++++++ .../main/scala/spark/metrics/sink/JmxSink.scala | 17 +++++++++++++++++ .../main/scala/spark/metrics/sink/Sink.scala | 17 +++++++++++++++++ .../scala/spark/metrics/source/JvmSource.scala | 17 +++++++++++++++++ .../scala/spark/metrics/source/Source.scala | 17 +++++++++++++++++ 8 files changed, 136 insertions(+) diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index ed505b0aa7..3e32e9c82f 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.metrics import java.util.Properties diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 2f87577ff3..fabddfb947 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.metrics import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index eaaac5d153..966ba37c20 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.metrics.sink import com.codahale.metrics.{ConsoleReporter, MetricRegistry} diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index aa5bff0d34..cb990afdef 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.metrics.sink import com.codahale.metrics.{CsvReporter, MetricRegistry} diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index 6a40885b78..ee04544c0e 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.metrics.sink import com.codahale.metrics.{JmxReporter, MetricRegistry} diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 3ffdcbdaba..dad1a7f0fe 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.metrics.sink trait Sink { diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 79f505079c..e771008557 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.metrics.source import com.codahale.metrics.MetricRegistry diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index 5607e2c40a..76199a004b 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.metrics.source import com.codahale.metrics.MetricRegistry From fefb03cbd79cd44447df31147e58f32ba0a466a8 Mon Sep 17 00:00:00 2001 From: cdshines Date: Wed, 31 Jul 2013 13:19:42 +0300 Subject: [PATCH 374/419] Eliminated code duplication, refactored to pattern-matching style Partitioner and PythonPartitioner --- core/src/main/scala/spark/Partitioner.scala | 14 +++-------- core/src/main/scala/spark/Utils.scala | 9 +++++++ .../spark/api/python/PythonPartitioner.scala | 25 ++++--------------- 3 files changed, 17 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 660af70d52..6035bc075e 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -65,17 +65,9 @@ object Partitioner { class HashPartitioner(partitions: Int) extends Partitioner { def numPartitions = partitions - def getPartition(key: Any): Int = { - if (key == null) { - return 0 - } else { - val mod = key.hashCode % partitions - if (mod < 0) { - mod + partitions - } else { - mod // Guard against negative hash codes - } - } + def getPartition(key: Any): Int = key match { + case null => 0 + case _ => Utils.nonNegativeMod(key.hashCode, numPartitions) } override def equals(other: Any): Boolean = other match { diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index e6a96a5ec1..c29933dfe9 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -756,4 +756,13 @@ private object Utils extends Logging { } return buf } + + /* Calculates 'x' modulo 'mod', takes to consideration sign of x, + * i.e. if 'x' is negative, than 'x' % 'mod' is negative too + * so function return (x % mod) + mod in that case. + */ + def nonNegativeMod(x: Int, mod: Int): Int = { + val rawMod = x % mod + rawMod + (if (x < 0 && rawMod != 0) mod else 0) + } } diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala index 31a719fbff..ac112b8c2c 100644 --- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala @@ -18,7 +18,7 @@ package spark.api.python import spark.Partitioner - +import spark.Utils import java.util.Arrays /** @@ -35,25 +35,10 @@ private[spark] class PythonPartitioner( val pyPartitionFunctionId: Long) extends Partitioner { - override def getPartition(key: Any): Int = { - if (key == null) { - return 0 - } - else { - val hashCode = { - if (key.isInstanceOf[Array[Byte]]) { - Arrays.hashCode(key.asInstanceOf[Array[Byte]]) - } else { - key.hashCode() - } - } - val mod = hashCode % numPartitions - if (mod < 0) { - mod + numPartitions - } else { - mod // Guard against negative hash codes - } - } + override def getPartition(key: Any): Int = key match { + case null => 0 + case key: Array[Byte] => Utils.nonNegativeMod(Arrays.hashCode(key), numPartitions) + case _ => Utils.nonNegativeMod(key.hashCode(), numPartitions) } override def equals(other: Any): Boolean = other match { From 12553e5c5534df1e5522825088b9a4a73d098854 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 31 Jul 2013 08:50:20 -0700 Subject: [PATCH 375/419] Simplified nonNegativeMod to match previous version --- core/src/main/scala/spark/Utils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c29933dfe9..ef598ae41b 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -596,7 +596,7 @@ private object Utils extends Logging { output.toString } - /** + /** * A regular expression to match classes of the "core" Spark API that we want to skip when * finding the call site of a method. */ @@ -763,6 +763,6 @@ private object Utils extends Logging { */ def nonNegativeMod(x: Int, mod: Int): Int = { val rawMod = x % mod - rawMod + (if (x < 0 && rawMod != 0) mod else 0) + rawMod + (if (rawMod < 0) mod else 0) } } From 0c6553714ab7b4f0daee59c0450f5004d69bd662 Mon Sep 17 00:00:00 2001 From: Roman Tkalenko Date: Wed, 31 Jul 2013 19:05:46 +0300 Subject: [PATCH 376/419] Refactored Vector.apply(length, initializer) replacing excessive code with library method (also removed unused variable ```ans``` as minor change) --- core/src/main/scala/spark/util/Vector.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/spark/util/Vector.scala index ed49386f18..a47cac3b96 100644 --- a/core/src/main/scala/spark/util/Vector.scala +++ b/core/src/main/scala/spark/util/Vector.scala @@ -73,7 +73,6 @@ class Vector(val elements: Array[Double]) extends Serializable { def += (other: Vector): Vector = { if (length != other.length) throw new IllegalArgumentException("Vectors of different length") - var ans = 0.0 var i = 0 while (i < length) { elements(i) += other(i) @@ -117,9 +116,7 @@ object Vector { def apply(elements: Double*) = new Vector(elements.toArray) def apply(length: Int, initializer: Int => Double): Vector = { - val elements = new Array[Double](length) - for (i <- 0 until length) - elements(i) = initializer(i) + val elements: Array[Double] = Array.tabulate(length)(initializer) return new Vector(elements) } From 9a815de4bfaf959fe1399d6550e561a0ec9f28c2 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Thu, 1 Aug 2013 00:36:47 +0800 Subject: [PATCH 377/419] write and read generation in ResultTask --- core/src/main/scala/spark/scheduler/ResultTask.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 361b1e6b91..1ced6f9524 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -118,6 +118,7 @@ private[spark] class ResultTask[T, U]( out.write(bytes) out.writeInt(partition) out.writeInt(outputId) + out.writeLong(generation) out.writeObject(split) } } @@ -132,6 +133,7 @@ private[spark] class ResultTask[T, U]( func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U] partition = in.readInt() val outputId = in.readInt() + generation = in.readLong() split = in.readObject().asInstanceOf[Partition] } } From 89da9d94b3f8d977360abeb659286ba7c2724825 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 31 Jul 2013 09:47:53 -0700 Subject: [PATCH 378/419] Add JSON path to master index page --- .../main/scala/spark/deploy/master/ui/IndexPage.scala | 9 ++++++++- .../main/scala/spark/deploy/master/ui/MasterWebUI.scala | 1 + 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index b05197c1b9..82a91cba03 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -25,15 +25,22 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import spark.deploy.{RequestMasterState, DeployWebUI, MasterState} +import spark.deploy.{JsonProtocol, RequestMasterState, DeployWebUI, MasterState} import spark.Utils import spark.ui.UIUtils import spark.deploy.master.{ApplicationInfo, WorkerInfo} +import net.liftweb.json.JsonAST.JValue private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.master implicit val timeout = parent.timeout + def renderJson(request: HttpServletRequest): JValue = { + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 30 seconds) + JsonProtocol.writeMasterState(state) + } + /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index dabc2d8dc7..31bdb7854e 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -61,6 +61,7 @@ class MasterWebUI(val master: ActorRef, requestedPort: Int) extends Logging { ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), ("/app/json", (request: HttpServletRequest) => applicationPage.renderJson(request)), ("/app", (request: HttpServletRequest) => applicationPage.render(request)), + ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), ("*", (request: HttpServletRequest) => indexPage.render(request)) ) From c61843a69fd50db66b01e9ef0fb2870baf51d351 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Jul 2013 18:54:35 -0700 Subject: [PATCH 379/419] Changed other LZF uses to use the compression codec interface. --- .../scala/spark/broadcast/HttpBroadcast.scala | 36 ++++++++++--------- .../spark/scheduler/ShuffleMapTask.scala | 15 ++------ .../scala/spark/streaming/Checkpoint.scala | 26 +++++++++----- 3 files changed, 39 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala index c565876950..138a8c21bc 100644 --- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala @@ -17,21 +17,20 @@ package spark.broadcast -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} - -import java.io._ -import java.net._ -import java.util.UUID +import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream} +import java.net.URL import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import spark._ +import spark.{HttpServer, Logging, SparkEnv, Utils} +import spark.io.CompressionCodec import spark.storage.StorageLevel -import util.{MetadataCleaner, TimeStampedHashSet} +import spark.util.{MetadataCleaner, TimeStampedHashSet} + private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) -extends Broadcast[T](id) with Logging with Serializable { + extends Broadcast[T](id) with Logging with Serializable { def value = value_ @@ -85,6 +84,7 @@ private object HttpBroadcast extends Logging { private val files = new TimeStampedHashSet[String] private val cleaner = new MetadataCleaner("HttpBroadcast", cleanup) + private lazy val compressionCodec = CompressionCodec.createCodec() def initialize(isDriver: Boolean) { synchronized { @@ -122,10 +122,12 @@ private object HttpBroadcast extends Logging { def write(id: Long, value: Any) { val file = new File(broadcastDir, "broadcast-" + id) - val out: OutputStream = if (compress) { - new LZFOutputStream(new FileOutputStream(file)) // Does its own buffering - } else { - new FastBufferedOutputStream(new FileOutputStream(file), bufferSize) + val out: OutputStream = { + if (compress) { + compressionCodec.compressedOutputStream(new FileOutputStream(file)) + } else { + new FastBufferedOutputStream(new FileOutputStream(file), bufferSize) + } } val ser = SparkEnv.get.serializer.newInstance() val serOut = ser.serializeStream(out) @@ -136,10 +138,12 @@ private object HttpBroadcast extends Logging { def read[T](id: Long): T = { val url = serverUri + "/broadcast-" + id - var in = if (compress) { - new LZFInputStream(new URL(url).openStream()) // Does its own buffering - } else { - new FastBufferedInputStream(new URL(url).openStream(), bufferSize) + val in = { + if (compress) { + compressionCodec.compressedInputStream(new URL(url).openStream()) + } else { + new FastBufferedInputStream(new URL(url).openStream(), bufferSize) + } } val ser = SparkEnv.get.serializer.newInstance() val serIn = ser.deserializeStream(in) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 1c25605f75..e3bb6d1e60 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -18,16 +18,9 @@ package spark.scheduler import java.io._ -import java.util.{HashMap => JHashMap} import java.util.zip.{GZIPInputStream, GZIPOutputStream} -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.collection.JavaConversions._ - -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream - -import com.ning.compress.lzf.LZFInputStream -import com.ning.compress.lzf.LZFOutputStream +import scala.collection.mutable.HashMap import spark._ import spark.executor.ShuffleWriteMetrics @@ -109,11 +102,7 @@ private[spark] class ShuffleMapTask( preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs)) } - var split = if (rdd == null) { - null - } else { - rdd.partitions(partition) - } + var split = if (rdd == null) null else rdd.partitions(partition) override def writeExternal(out: ObjectOutput) { RDDCheckpointData.synchronized { diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 1e4c1e3742..070d930b5e 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -17,16 +17,17 @@ package spark.streaming -import spark.{Logging, Utils} - -import org.apache.hadoop.fs.{FileUtil, Path} -import org.apache.hadoop.conf.Configuration - import java.io._ -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import java.util.concurrent.Executors import java.util.concurrent.RejectedExecutionException +import org.apache.hadoop.fs.Path +import org.apache.hadoop.conf.Configuration + +import spark.Logging +import spark.io.CompressionCodec + + private[streaming] class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Logging with Serializable { @@ -49,6 +50,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) } } + /** * Convenience class to speed up the writing of graph checkpoint to file */ @@ -66,6 +68,8 @@ class CheckpointWriter(checkpointDir: String) extends Logging { val maxAttempts = 3 val executor = Executors.newFixedThreadPool(1) + private val compressionCodec = CompressionCodec.createCodec() + // Removed code which validates whether there is only one CheckpointWriter per path 'file' since // I did not notice any errors - reintroduce it ? @@ -103,7 +107,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging { def write(checkpoint: Checkpoint) { val bos = new ByteArrayOutputStream() - val zos = new LZFOutputStream(bos) + val zos = compressionCodec.compressedOutputStream(bos) val oos = new ObjectOutputStream(zos) oos.writeObject(checkpoint) oos.close() @@ -137,6 +141,8 @@ object CheckpointReader extends Logging { val fs = new Path(path).getFileSystem(new Configuration()) val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk")) + val compressionCodec = CompressionCodec.createCodec() + attempts.foreach(file => { if (fs.exists(file)) { logInfo("Attempting to load checkpoint from file '" + file + "'") @@ -147,7 +153,7 @@ object CheckpointReader extends Logging { // of ObjectInputStream is used to explicitly use the current thread's default class // loader to find and load classes. This is a well know Java issue and has popped up // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) - val zis = new LZFInputStream(fis) + val zis = compressionCodec.compressedInputStream(fis) val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader) val cp = ois.readObject.asInstanceOf[Checkpoint] ois.close() @@ -170,7 +176,9 @@ object CheckpointReader extends Logging { } private[streaming] -class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) extends ObjectInputStream(inputStream_) { +class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) + extends ObjectInputStream(inputStream_) { + override def resolveClass(desc: ObjectStreamClass): Class[_] = { try { return loader.loadClass(desc.getName()) From 49e6344142b9e87cf220d93c41ae6d35965ea88e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 31 Jul 2013 11:17:59 -0700 Subject: [PATCH 380/419] Removed master URL from job UI, reduced heading size of basic spark pages --- core/src/main/scala/spark/ui/UIUtils.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index e33c80282a..18a3bf534c 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -74,7 +74,6 @@ private[spark] object UIUtils {
    • Application: {sc.appName}
    • -
    • Master: {sc.master}
    • Executors: {sc.getExecutorStorageStatus.size}
    @@ -117,9 +116,9 @@ private[spark] object UIUtils {
    -

    +

    {title} -

    +
    {content} From 9a444cffe74374f0d764d1ed8197423e40529f24 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 31 Jul 2013 11:28:39 -0700 Subject: [PATCH 381/419] Use the Char version of split() instead of the String one for efficiency --- mllib/src/main/scala/spark/mllib/util/MLUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala index b5e564df6d..25d9673004 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -38,9 +38,9 @@ object MLUtils { */ def loadLabeledData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { sc.textFile(dir).map { line => - val parts = line.split(",") + val parts = line.split(',') val label = parts(0).toDouble - val features = parts(1).trim().split(" ").map(_.toDouble) + val features = parts(1).trim().split(' ').map(_.toDouble) (label, features) } } From c453967f9aa5946a263fa1364587e7fca88aae45 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 31 Jul 2013 11:57:50 -0700 Subject: [PATCH 382/419] Reduced size of heading --- core/src/main/scala/spark/ui/UIUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index 18a3bf534c..cff26d3168 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -116,9 +116,9 @@ private[spark] object UIUtils {
    -

    +

    {title} -

    +
    {content} From 4692ea48920cb7cebd2cf8cacc43382168801722 Mon Sep 17 00:00:00 2001 From: Benjamin Hindman Date: Wed, 31 Jul 2013 12:29:42 -0700 Subject: [PATCH 383/419] Used 'uri.split('/').last' instead of 'new File(uri).getName()'. --- .../spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../scala/spark/scheduler/mesos/MesosSchedulerBackend.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index f75244a9ac..6ebbb5ec9b 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -131,7 +131,7 @@ private[spark] class CoarseMesosSchedulerBackend( } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". - val basename = new File(uri).getName().split('.')(0) + val basename = uri.split('/').last.split('.').head command.setValue("cd %s*; ./run spark.executor.StandaloneExecutorBackend %s %s %s %d".format( basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 51b780ac72..f6069a5775 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -104,7 +104,7 @@ private[spark] class MesosSchedulerBackend( } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". - val basename = new File(uri).getName().split('.')(0) + val basename = uri.split('/').last.split('.').head command.setValue("cd %s*; ./spark-executor".format(basename)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } From 529ac81195505f59228fd212a5bad154ab316683 Mon Sep 17 00:00:00 2001 From: Benjamin Hindman Date: Wed, 31 Jul 2013 12:49:11 -0700 Subject: [PATCH 384/419] Do not try and use 'scala' in 'run' from within a "release". --- run | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/run b/run index 4cffda4708..0a440627a1 100755 --- a/run +++ b/run @@ -72,7 +72,10 @@ esac # hard to kill the child with stuff like Process.destroy(). However, for # the Spark shell, the wrapper is necessary to properly reset the terminal # when we exit, so we allow it to set a variable to launch with scala. -if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then +# We still fall back on java for the shell if this is a "release" created +# from make-distribution.sh since it's possible scala is not installed +# but we have everything we need to run the shell. +if [[ "$SPARK_LAUNCH_WITH_SCALA" == "1" && ! -f "$FWDIR/RELEASE" ]]; then if [ "$SCALA_HOME" ]; then RUNNER="${SCALA_HOME}/bin/scala" else From f607ffb9e1f799d73818f1d37c633007a6b900fb Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 31 Jul 2013 14:31:07 -0700 Subject: [PATCH 385/419] Added data generator for K-means Also made it possible to specify the number of runs in KMeans.main(). --- .../scala/spark/mllib/clustering/KMeans.scala | 7 +- .../mllib/util/KMeansDataGenerator.scala | 80 +++++++++++++++++++ 2 files changed, 84 insertions(+), 3 deletions(-) create mode 100644 mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala index d875d6de50..a2ed42d7a5 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala @@ -315,14 +315,15 @@ object KMeans { } def main(args: Array[String]) { - if (args.length != 4) { - println("Usage: KMeans ") + if (args.length < 4) { + println("Usage: KMeans []") System.exit(1) } val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) + val runs = if (args.length >= 5) args(4).toInt else 1 val sc = new SparkContext(master, "KMeans") val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) - val model = KMeans.train(data, k, iters) + val model = KMeans.train(data, k, iters, runs) val cost = model.computeCost(data) println("Cluster centers:") for (c <- model.clusterCenters) { diff --git a/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala new file mode 100644 index 0000000000..8f95cf7479 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.util + +import scala.util.Random + +import spark.{RDD, SparkContext} + +object KMeansDataGenerator { + + /** + * Generate an RDD containing test data for KMeans. This function chooses k cluster centers + * from a d-dimensional Gaussian distribution scaled by factor r, then creates a Gaussian + * cluster with scale 1 around each center. + * + * @param sc SparkContext to use for creating the RDD + * @param numPoints Number of points that will be contained in the RDD + * @param k Number of clusters + * @param d Number of dimensions + * @parak r Scaling factor for the distribution of the initial centers + * @param numPartitions Number of partitions of the generated RDD; default 2 + */ + def generateKMeansRDD( + sc: SparkContext, + numPoints: Int, + k: Int, + d: Int, + r: Double, + numPartitions: Int = 2) + : RDD[Array[Double]] = + { + // First, generate some centers + val rand = new Random(42) + val centers = Array.fill(k)(Array.fill(d)(rand.nextGaussian() * r)) + // Then generate points around each center + sc.parallelize(0 until numPoints, numPartitions).map { idx => + val center = centers(idx % k) + val rand2 = new Random(42 + idx) + Array.tabulate(d)(i => center(i) + rand2.nextGaussian()) + } + } + + def main(args: Array[String]) { + if (args.length < 6) { + println("Usage: KMeansGenerator " + + " []") + System.exit(1) + } + + val sparkMaster = args(0) + val outputPath = args(1) + val numPoints = args(2).toInt + val k = args(3).toInt + val d = args(4).toInt + val r = args(5).toDouble + val parts = if (args.length >= 7) args(6).toInt else 2 + + val sc = new SparkContext(sparkMaster, "KMeansDataGenerator") + val data = generateKMeansRDD(sc, numPoints, k, d, r, parts) + data.map(_.mkString(" ")).saveAsTextFile(outputPath) + + System.exit(0) + } +} + From a6f43a97d15e08f0d2c76b8b790f7e5635f5b2e6 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Wed, 31 Jul 2013 14:45:53 -0700 Subject: [PATCH 386/419] SPARK-842. Maven assembly is including examples libs and dependencies --- assembly/pom.xml | 24 ++++++------------------ assembly/src/main/assembly/assembly.xml | 2 +- 2 files changed, 7 insertions(+), 19 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 1382539f24..cc5a4875af 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -62,43 +62,31 @@ org.spark-project spark-core ${classifier.name} - 0.8.0-SNAPSHOT + ${project.version}
    org.spark-project spark-bagel ${classifier.name} - 0.8.0-SNAPSHOT + ${project.version} org.spark-project - spark-examples + spark-mllib ${classifier.name} - 0.8.0-SNAPSHOT - - - org.spark-project - spark-examples - javadoc - 0.8.0-SNAPSHOT - - - org.spark-project - spark-examples - sources - 0.8.0-SNAPSHOT + ${project.version} org.spark-project spark-repl ${classifier.name} - 0.8.0-SNAPSHOT + ${project.version} org.spark-project spark-streaming ${classifier.name} - 0.8.0-SNAPSHOT + ${project.version} \ No newline at end of file diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml index dd05f35f1f..14485b7181 100644 --- a/assembly/src/main/assembly/assembly.xml +++ b/assembly/src/main/assembly/assembly.xml @@ -49,7 +49,7 @@ org.spark-project:*:jar - org.spark-project:spark-dist:jar + org.spark-project:spark-assembly:jar From 52dba89261ee6dddafff5c746322980567252843 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 31 Jul 2013 23:08:12 -0700 Subject: [PATCH 387/419] Turn on caching in KMeans.main --- mllib/src/main/scala/spark/mllib/clustering/KMeans.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala index a2ed42d7a5..b402c71ed2 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala @@ -322,7 +322,7 @@ object KMeans { val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) val runs = if (args.length >= 5) args(4).toInt else 1 val sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) + val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)).cache() val model = KMeans.train(data, k, iters, runs) val cost = model.computeCost(data) println("Cluster centers:") From 3b5a11e76563a270d694bdcf8afac0b0a515a631 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 1 Aug 2013 19:37:15 +0800 Subject: [PATCH 388/419] change function name "setName" to "setProperties" as "setName" is also member of Thread class --- .../src/main/scala/spark/ui/UIWorkloadGenerator.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index c738cc4065..4fbb503e5c 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -32,6 +32,8 @@ private[spark] object UIWorkloadGenerator { val NUM_PARTITIONS = 100 val INTER_JOB_WAIT_MS = 500 + + def main(args: Array[String]) { if (args.length < 2) { println("usage: ./run spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") @@ -48,12 +50,13 @@ private[spark] object UIWorkloadGenerator { // NOTE: Right now there is no easy way for us to show spark.job.annotation for a given phase, // but we pass it here anyways since it will be useful once we do. - def setName(s: String) = { + def setProperties(s: String) = { if(schedulingMode == SchedulingMode.FAIR) { - sc.addLocalProperties("spark.scheduler.cluster.fair.pool",desc) + sc.addLocalProperties("spark.scheduler.cluster.fair.pool", s) } sc.addLocalProperties("spark.job.annotation", s) } + val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) def nextFloat() = (new Random()).nextFloat() @@ -87,9 +90,9 @@ private[spark] object UIWorkloadGenerator { while (true) { for ((desc, job) <- jobs) { new Thread { - override def run() { + override def run() { try { - setName(desc) + setProperties(desc) job() println("Job funished: " + desc) } catch { From d58502a1562bbfb1bb4e517ebcc8239efd639297 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 1 Aug 2013 23:21:41 +0800 Subject: [PATCH 389/419] fix bug of spark "SubmitStage" listener as unit test error --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 88c1b7baff..89c51a44c9 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -510,6 +510,12 @@ class DAGScheduler( tasks += new ResultTask(stage.id, stage.rdd, job.func, partition, locs, id) } } + // must be run listener before possible NotSerializableException + // should be "StageSubmitted" first and then "JobEnded" + val properties = idToActiveJob(stage.priority).properties + sparkListeners.foreach(_.onStageSubmitted( + SparkListenerStageSubmitted(stage, tasks.size, properties))) + if (tasks.size > 0) { // Preemptively serialize a task to make sure it can be serialized. We are catching this // exception here because it would be fairly hard to catch the non-serializable exception @@ -523,9 +529,7 @@ class DAGScheduler( running -= stage return } - val properties = idToActiveJob(stage.priority).properties - sparkListeners.foreach(_.onStageSubmitted( - SparkListenerStageSubmitted(stage, tasks.size, properties))) + logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) From ffc034e4fbb6d63825626f555b2089c0389d0075 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 10:39:56 -0700 Subject: [PATCH 390/419] Import cleanup --- .../main/scala/spark/deploy/master/ui/IndexPage.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 82a91cba03..df4c00fc97 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -17,19 +17,20 @@ package spark.deploy.master.ui +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ -import javax.servlet.http.HttpServletRequest - -import scala.xml.Node +import net.liftweb.json.JsonAST.JValue import spark.deploy.{JsonProtocol, RequestMasterState, DeployWebUI, MasterState} import spark.Utils import spark.ui.UIUtils import spark.deploy.master.{ApplicationInfo, WorkerInfo} -import net.liftweb.json.JsonAST.JValue private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.master From 9177bea2b4274c6772e85a78e9264486ced01893 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 10:42:50 -0700 Subject: [PATCH 391/419] Removing extra imports --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index a15558c05e..79fdb21024 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -25,10 +25,6 @@ import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ -import akka.dispatch.Await -import akka.pattern.ask -import akka.util.duration._ - import net.liftweb.json.JsonAST.JValue import spark.Utils From 5faac7f4f3678d424ea1e40a109996f3caba2142 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 13:09:20 -0700 Subject: [PATCH 392/419] Minor style fixes --- core/src/main/scala/spark/SparkContext.scala | 2 +- core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 0d1f9fa8d4..2e2a699708 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -579,7 +579,7 @@ class SparkContext( /** * Return pools for fair scheduler - * TODO(xiajunluan):now, we have not taken nested pools into account + * TODO(xiajunluan): We should take nested pools into account */ def getPools: ArrayBuffer[Schedulable] = { taskScheduler.rootPool.schedulableQueue diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 3832c5d33c..dc3aef8f28 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -41,7 +41,6 @@ private[spark] class JobProgressUI(val sc: SparkContext) { def listener = _listener.get val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) From cfcd77b5da7d8c453b525c1862cba158eddc5170 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 13:31:56 -0700 Subject: [PATCH 393/419] Increasing inter job arrival --- core/src/main/scala/spark/ui/UIWorkloadGenerator.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 4fbb503e5c..3ac35085eb 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -30,9 +30,7 @@ import spark.scheduler.cluster.SchedulingMode.SchedulingMode */ private[spark] object UIWorkloadGenerator { val NUM_PARTITIONS = 100 - val INTER_JOB_WAIT_MS = 500 - - + val INTER_JOB_WAIT_MS = 5000 def main(args: Array[String]) { if (args.length < 2) { From b10199413a1e9193c605dc17de276c78b578e545 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 13:37:22 -0700 Subject: [PATCH 394/419] Slight refactoring to SparkContext functions --- core/src/main/scala/spark/SparkContext.scala | 13 ++++++++----- core/src/main/scala/spark/ui/jobs/PoolTable.scala | 4 ++-- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 2e2a699708..97e1aaf49e 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -581,10 +581,17 @@ class SparkContext( * Return pools for fair scheduler * TODO(xiajunluan): We should take nested pools into account */ - def getPools: ArrayBuffer[Schedulable] = { + def getAllPools: ArrayBuffer[Schedulable] = { taskScheduler.rootPool.schedulableQueue } + /** + * Return the pool associated with the given name, if one exists + */ + def getPoolForName(pool: String): Option[Schedulable] = { + taskScheduler.rootPool.schedulableNameToSchedulable.get(pool) + } + /** * Return current scheduling mode */ @@ -592,10 +599,6 @@ class SparkContext( taskScheduler.schedulingMode } - def getPoolNameToPool: HashMap[String, Schedulable] = { - taskScheduler.rootPool.schedulableNameToSchedulable - } - /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala index 8788ed8bc1..29061199df 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -38,7 +38,7 @@ private[spark] class FIFOSource() extends PoolSource { */ private[spark] class FairSource(sc: SparkContext) extends PoolSource { def getPools: Seq[Schedulable] = { - sc.getPools.toSeq + sc.getAllPools.toSeq } } @@ -48,7 +48,7 @@ private[spark] class FairSource(sc: SparkContext) extends PoolSource { private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource { def getPools: Seq[Schedulable] = { val pools = HashSet[Schedulable]() - pools += sc.getPoolNameToPool(poolName) + pools += sc.getPoolForName(poolName).get pools.toSeq } } From 87fd321a5a12ccea9d5593a4c43aaadfa44855bd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 1 Aug 2013 14:11:18 -0700 Subject: [PATCH 395/419] Minor refactoring and code cleanup --- .../main/scala/spark/ui/jobs/IndexPage.scala | 27 +++++----- .../scala/spark/ui/jobs/JobProgressUI.scala | 12 ----- .../main/scala/spark/ui/jobs/PoolPage.scala | 7 ++- .../main/scala/spark/ui/jobs/PoolTable.scala | 52 ++----------------- .../main/scala/spark/ui/jobs/StageTable.scala | 51 ++++-------------- 5 files changed, 28 insertions(+), 121 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 4ad787565d..b0d057afa1 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -17,16 +17,11 @@ package spark.ui.jobs -import java.util.Date - import javax.servlet.http.HttpServletRequest -import scala.Some import scala.xml.{NodeSeq, Node} -import spark.scheduler.cluster.TaskInfo -import spark.scheduler.Stage -import spark.storage.StorageLevel +import spark.scheduler.cluster.SchedulingMode import spark.ui.Page._ import spark.ui.UIUtils._ import spark.Utils @@ -50,7 +45,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStagesTable = new StageTable(completedStages, parent) val failedStagesTable = new StageTable(failedStages, parent) - val poolTable = new PoolTable(parent.stagePagePoolSource, listener) + val poolTable = new PoolTable(listener.sc.getAllPools, listener) val summary: NodeSeq =
      @@ -79,13 +74,17 @@ private[spark] class IndexPage(parent: JobProgressUI) {
    val content = summary ++ -

    Pools

    ++ poolTable.toNodeSeq ++ -

    Active Stages : {activeStages.size}

    ++ - activeStagesTable.toNodeSeq++ -

    Completed Stages : {completedStages.size}

    ++ - completedStagesTable.toNodeSeq++ -

    Failed Stages : {failedStages.size}

    ++ - failedStagesTable.toNodeSeq + {if (listener.sc.getSchedulingMode == SchedulingMode.FAIR) { +

    Pools

    ++ poolTable.toNodeSeq + } else { + Seq() + }} ++ +

    Active Stages : {activeStages.size}

    ++ + activeStagesTable.toNodeSeq++ +

    Completed Stages : {completedStages.size}

    ++ + completedStagesTable.toNodeSeq++ +

    Failed Stages : {failedStages.size}

    ++ + failedStagesTable.toNodeSeq headerSparkPage(content, parent.sc, "Spark Stages", Jobs) } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index dc3aef8f28..c83f102ff3 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -45,20 +45,8 @@ private[spark] class JobProgressUI(val sc: SparkContext) { private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) - var stagePoolInfo: StagePoolInfo = null - var stagePagePoolSource: PoolSource = null - def start() { _listener = Some(new JobProgressListener(sc)) - sc.getSchedulingMode match { - case SchedulingMode.FIFO => - stagePoolInfo = new FIFOStagePoolInfo() - stagePagePoolSource = new FIFOSource() - case SchedulingMode.FAIR => - stagePoolInfo = new FairStagePoolInfo(listener) - stagePagePoolSource = new FairSource(sc) - } - sc.addSparkListener(listener) } diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala index 37d4f8fa6b..ee5a6a6a48 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -17,12 +17,11 @@ private[spark] class PoolPage(parent: JobProgressUI) { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages val activeStages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]).toSeq - - val poolDetailPoolSource = new PoolDetailSource(parent.sc, poolName) - val poolTable = new PoolTable(poolDetailPoolSource, listener) - val activeStagesTable = new StageTable(activeStages, parent) + val pool = listener.sc.getPoolForName(poolName).get + val poolTable = new PoolTable(Seq(pool), listener) + val content =

    Pool

    ++ poolTable.toNodeSeq() ++

    Active Stages : {activeStages.size}

    ++ activeStagesTable.toNodeSeq() diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala index 29061199df..9cfe0d68f0 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -1,65 +1,19 @@ package spark.ui.jobs -import java.util.Date - -import javax.servlet.http.HttpServletRequest - -import scala.Some -import scala.xml.{NodeSeq, Node} +import scala.xml.Node import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import spark.SparkContext import spark.scheduler.Stage -import spark.ui.UIUtils._ -import spark.ui.Page._ -import spark.storage.StorageLevel import spark.scheduler.cluster.Schedulable -/* - * Interface for get pools seq showing on Index or pool detail page - */ - -private[spark] trait PoolSource { - def getPools: Seq[Schedulable] -} - -/* - * Pool source for FIFO scheduler algorithm on Index page - */ -private[spark] class FIFOSource() extends PoolSource { - def getPools: Seq[Schedulable] = { - Seq[Schedulable]() - } -} - -/* - * Pool source for Fair scheduler algorithm on Index page - */ -private[spark] class FairSource(sc: SparkContext) extends PoolSource { - def getPools: Seq[Schedulable] = { - sc.getAllPools.toSeq - } -} - -/* - * specific pool info for pool detail page - */ -private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource { - def getPools: Seq[Schedulable] = { - val pools = HashSet[Schedulable]() - pools += sc.getPoolForName(poolName).get - pools.toSeq - } -} - /** Table showing list of pools */ -private[spark] class PoolTable(poolSource: PoolSource, listener: JobProgressListener) { +private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressListener) { var poolToActiveStages: HashMap[String, HashSet[Stage]] = listener.poolToActiveStages def toNodeSeq(): Seq[Node] = { - poolTable(poolRow, poolSource.getPools) + poolTable(poolRow, pools) } // pool tables diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index e18b70f0b9..3257f4e360 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -10,51 +10,20 @@ import scala.xml.{NodeSeq, Node} import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import spark.scheduler.cluster.TaskInfo +import spark.scheduler.cluster.{SchedulingMode, TaskInfo} import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ import spark.Utils import spark.storage.StorageLevel -/* - * Interface to get stage's pool name - */ -private[spark] trait StagePoolInfo { - def getStagePoolName(s: Stage): String - - def hasHref: Boolean -} - -/* - * For FIFO scheduler algorithm, just show "N/A" and its link status is false - */ -private[spark] class FIFOStagePoolInfo extends StagePoolInfo { - def getStagePoolName(s: Stage): String = "N/A" - - def hasHref: Boolean = false -} - -/* - * For Fair scheduler algorithm, show its pool name and pool detail link status is true - */ -private[spark] class FairStagePoolInfo(listener: JobProgressListener) extends StagePoolInfo { - def getStagePoolName(s: Stage): String = { - listener.stageToPool(s) - } - - def hasHref: Boolean = true -} - /** Page showing list of all ongoing and recently finished stages */ -private[spark] class StageTable( - val stages: Seq[Stage], - val parent: JobProgressUI) { +private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressUI) { val listener = parent.listener val dateFmt = parent.dateFmt - var stagePoolInfo = parent.stagePoolInfo - + val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR + def toNodeSeq(): Seq[Node] = { stageTable(stageRow, stages) } @@ -64,7 +33,7 @@ private[spark] class StageTable(
    (none) If you use Kryo serialization, set this class to register your custom classes with Kryo. - You need to set it to a class that extends + You need to set it to a class that extends spark.KryoRegistrator). See the tuning guide for more details.
    spark.io.compression.codecspark.io.SnappyCompressionCodec + The compression codec class to use for various compressions. By default, Spark provides two + codecs: spark.io.LZFCompressionCodec and spark.io.SnappyCompressionCodec. +
    spark.io.compression.snappy.block.size32768 + Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec is used. +
    spark.reducer.maxMbInFlight 48
    - + {if (isFairScheduler) {} else {}} @@ -116,15 +85,13 @@ private[spark] class StageTable( val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions - val poolName = stagePoolInfo.getStagePoolName(s) + val poolName = listener.stageToPool.get(s) - + {if (isFairScheduler) { + } + } {if (isFairScheduler) {} else {}} - + @@ -87,13 +87,17 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU val poolName = listener.stageToPool.get(s) + val nameLink = {s.name} + val description = listener.stageToDescription.get(s) + .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) + {if (isFairScheduler) { } } - - + + diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala index 14bb58731b..66fd59e8bb 100644 --- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -73,7 +73,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { TaskThreadInfo.threadToStarted(threadIndex) = new CountDownLatch(1) new Thread { if (poolName != null) { - sc.addLocalProperties("spark.scheduler.cluster.fair.pool",poolName) + sc.addLocalProperty("spark.scheduler.cluster.fair.pool", poolName) } override def run() { val ans = nums.map(number => { From abfa9e6f708b46894be1972f46efe542578a30f1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 2 Aug 2013 16:17:32 -0700 Subject: [PATCH 403/419] Increase Kryo buffer size in ALS since some arrays become big --- mllib/src/main/scala/spark/mllib/recommendation/ALS.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 7281b2fcb9..6ecf0151a1 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -418,6 +418,7 @@ object ALS { System.setProperty("spark.serializer", "spark.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) System.setProperty("spark.kryo.referenceTracking", "false") + System.setProperty("spark.kryoserializer.buffer.mb", "8") System.setProperty("spark.locality.wait", "10000") val sc = new SparkContext(master, "ALS") val ratings = sc.textFile(ratingsFile).map { line => From 4ab4df5edbc1bded810a8a3e1dfc7f8ae40a7c30 Mon Sep 17 00:00:00 2001 From: Ginger Smith Date: Fri, 2 Aug 2013 22:22:36 -0700 Subject: [PATCH 404/419] adding matrix factorization data generator --- .../spark/mllib/util/MFDataGenerator.scala | 105 ++++++++++++++++++ 1 file changed, 105 insertions(+) create mode 100644 mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala new file mode 100644 index 0000000000..8637d27cd0 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.recommendation + +import scala.util.Random + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.util.MLUtils + + +object MFDataGenerator{ + + /** + * Generate RDD(s) containing data for Matrix Factorization. This function chooses + * positive labels with probability `probOne` and scales positive examples by `eps`. + * + * @param sc SparkContext to use for creating the RDD. + * @param outputPath Directory to save output. + * @param m Number of rows in data matrix. + * @param n Number of columns in data matrix. + * @param rank Underlying rank of data matrix. + * @param tr_samp_fact Oversampling factor. + * @param noise Boolean value - whether to add gaussian noise to training data. + * @param sigma Standard deviation of added gaussian noise. + * @param test Boolean value - whether to create testing RDD. + * @param te_samp_fact Percentage of training data to use as test data. + */ + + def main(args: Array[String]) { + if (args.length != 10) { + println("Usage: MFGenerator " + + " ") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val m: Int = if (args.length > 2) args(2).toInt else 100 + val n: Int = if (args.length > 3) args(3).toInt else 100 + val rank: Int = if (args.length > 4) args(4).toInt else 10 + val tr_samp_fact: Double = if (args.length > 5) args(5).toDouble else 1.0 + val noise: Boolean = if (args.length > 6) args(6).toBoolean else false + val sigma: Double = if (args.length > 7) args(7).toDouble else 0.1 + val test: Boolean = if (args.length > 8) args(8).toBoolean else false + val te_samp_fact: Double = if (args.length > 9) args(9).toDouble else 0.1 + + val sc = new SparkContext(sparkMaster, "MFDataGenerator") + + val A = DoubleMatrix.randn(m,rank) + val B = DoubleMatrix.randn(rank,n) + val z = 1/(scala.math.sqrt(scala.math.sqrt(rank))) + A.mmuli(z) + B.mmuli(z) + val fullData = A.mmul(B) + + val df = rank*(m+n-rank) + val sampsize = scala.math.min(scala.math.round(tr_samp_fact*df), scala.math.round(.99*m*n)).toInt + val rand = new Random() + val mn = m*n + val shuffled = rand.shuffle(1 to mn toIterable) + + val omega = shuffled.slice(0,sampsize) + val ordered = omega.sortWith(_ < _).toArray + val trainData: RDD[(Int, Int, Double)] = sc.parallelize(ordered) + .map(x => (fullData.indexRows(x-1),fullData.indexColumns(x-1),fullData.get(x-1))) + + // optionally add gaussian noise + if(noise){ + trainData.map(x => (x._1,x._2,x._3+rand.nextGaussian*sigma)) + } + + trainData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) + + // optionally generate testing data + if(test){ + val test_sampsize = scala.math + .min(scala.math.round(sampsize*te_samp_fact),scala.math.round(mn-sampsize)) + .toInt + val test_omega = shuffled.slice(sampsize,sampsize+test_sampsize) + val test_ordered = test_omega.sortWith(_ < _).toArray + val testData: RDD[(Int, Int, Double)] = sc.parallelize(test_ordered) + .map(x=> (fullData.indexRows(x-1),fullData.indexColumns(x-1),fullData.get(x-1))) + testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) + } + + sc.stop() + } +} \ No newline at end of file From d93d5fcaacc22f837c861bddbb54928b8121bdfb Mon Sep 17 00:00:00 2001 From: Bill Zhao Date: Mon, 5 Aug 2013 10:09:03 -0700 Subject: [PATCH 405/419] SPARK-850: Give better error message on the console --- .../main/scala/spark/scheduler/cluster/ClusterScheduler.scala | 3 ++- project/SparkBuild.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index d92dd3de8b..96568e0d27 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -206,7 +206,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def run() { if (!hasLaunchedTask) { logWarning("Initial job has not accepted any resources; " + - "check your cluster UI to ensure that workers are registered") + "check your cluster UI to ensure that workers are registered " + + "and have sufficient memory") } else { this.cancel() } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c822f49e78..4a9f3de8c9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -156,7 +156,7 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( - "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", + // "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "Spray Repository" at "http://repo.spray.cc/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), From 8c8947e2b66169dddb828b801ffaa43cc400b8a5 Mon Sep 17 00:00:00 2001 From: Ginger Smith Date: Mon, 5 Aug 2013 11:22:18 -0700 Subject: [PATCH 406/419] fixing formatting --- .../spark/mllib/util/MFDataGenerator.scala | 39 +++++++++++-------- 1 file changed, 23 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala index 8637d27cd0..1d2b5c89f0 100644 --- a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala @@ -24,25 +24,32 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.util.MLUtils +/** +* Generate RDD(s) containing data for Matrix Factorization. +* +* This method samples training entries according to the oversampling factor +* 'tr_samp_fact', which is a multiplicative factor of the number of +* degrees of freedom of the matrix: rank*(m+n-rank). +* +* It optionally samples entries for a testing matrix using +* 'te_samp_fact', the percentage of the number of training entries +* to use for testing. +* +* This method takes the following inputs: +* sparkMaster (String) The master URL. +* outputPath (String) Directory to save output. +* m (Int) Number of rows in data matrix. +* n (Int) Number of columns in data matrix. +* rank (Int) Underlying rank of data matrix. +* tr_samp_fact (Double) Oversampling factor. +* noise (Boolean) Whether to add gaussian noise to training data. +* sigma (Double) Standard deviation of added gaussian noise. +* test (Boolean) Whether to create testing RDD. +* te_samp_fact (Double) Percentage of training data to use as test data. +*/ object MFDataGenerator{ - /** - * Generate RDD(s) containing data for Matrix Factorization. This function chooses - * positive labels with probability `probOne` and scales positive examples by `eps`. - * - * @param sc SparkContext to use for creating the RDD. - * @param outputPath Directory to save output. - * @param m Number of rows in data matrix. - * @param n Number of columns in data matrix. - * @param rank Underlying rank of data matrix. - * @param tr_samp_fact Oversampling factor. - * @param noise Boolean value - whether to add gaussian noise to training data. - * @param sigma Standard deviation of added gaussian noise. - * @param test Boolean value - whether to create testing RDD. - * @param te_samp_fact Percentage of training data to use as test data. - */ - def main(args: Array[String]) { if (args.length != 10) { println("Usage: MFGenerator " + From 87134b3648c3e01aa2f1fff28751f3625c6434a2 Mon Sep 17 00:00:00 2001 From: Bill Zhao Date: Mon, 5 Aug 2013 11:55:35 -0700 Subject: [PATCH 407/419] SPARK-850: give better console message --- .../main/scala/spark/scheduler/cluster/ClusterScheduler.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index d92dd3de8b..96568e0d27 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -206,7 +206,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) override def run() { if (!hasLaunchedTask) { logWarning("Initial job has not accepted any resources; " + - "check your cluster UI to ensure that workers are registered") + "check your cluster UI to ensure that workers are registered " + + "and have sufficient memory") } else { this.cancel() } From 33b9b155defe915d69de25d00da5d2dbdbb71eb5 Mon Sep 17 00:00:00 2001 From: Bill Zhao Date: Mon, 5 Aug 2013 12:02:36 -0700 Subject: [PATCH 408/419] JBoss repository working now --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4a9f3de8c9..c822f49e78 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -156,7 +156,7 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", resolvers ++= Seq( - // "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", + "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "Spray Repository" at "http://repo.spray.cc/", "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), From e8bec8365f6598affd0335eae82b093acf4671da Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Wed, 31 Jul 2013 15:52:05 -0700 Subject: [PATCH 409/419] Only reduce the number of cores once when removing an executor --- .../main/scala/spark/deploy/master/ApplicationInfo.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 79687df614..6dd2f06126 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -52,8 +52,10 @@ private[spark] class ApplicationInfo( } def removeExecutor(exec: ExecutorInfo) { - executors -= exec.id - coresGranted -= exec.cores + if (executors.contains(exec.id)) { + executors -= exec.id + coresGranted -= exec.cores + } } def coresLeft: Int = desc.maxCores - coresGranted From cdd1af562ef3fb480f2e98300e3d463657c09681 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 1 Aug 2013 15:14:39 -0700 Subject: [PATCH 410/419] Timeout zombie workers --- .../scala/spark/deploy/master/Master.scala | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 0aed4b9802..b50613f866 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -39,7 +39,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt - + val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt + var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] val idToWorker = new HashMap[String, WorkerInfo] @@ -337,12 +338,17 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act /** Check for, and remove, any timed-out workers */ def timeOutDeadWorkers() { // Copy the workers into an array so we don't modify the hashset while iterating through it - val expirationTime = System.currentTimeMillis() - WORKER_TIMEOUT - val toRemove = workers.filter(_.lastHeartbeat < expirationTime).toArray + val currentTime = System.currentTimeMillis() + val toRemove = workers.filter(_.lastHeartbeat < currentTime - WORKER_TIMEOUT).toArray for (worker <- toRemove) { - logWarning("Removing %s because we got no heartbeat in %d seconds".format( - worker.id, WORKER_TIMEOUT)) - removeWorker(worker) + if (worker.state != WorkerState.DEAD) { + logWarning("Removing %s because we got no heartbeat in %d seconds".format( + worker.id, WORKER_TIMEOUT)) + removeWorker(worker) + } else { + if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) + workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it + } } } } From 37ccf9301a427095d5bf6a35447f5871e12cdb35 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 1 Aug 2013 15:33:42 -0700 Subject: [PATCH 411/419] milliseconds -> seconds in timeOutDeadWorkers logging --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index b50613f866..bd7924c71d 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -343,7 +343,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act for (worker <- toRemove) { if (worker.state != WorkerState.DEAD) { logWarning("Removing %s because we got no heartbeat in %d seconds".format( - worker.id, WORKER_TIMEOUT)) + worker.id, WORKER_TIMEOUT/1000)) removeWorker(worker) } else { if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) From 35d8f5ee521dc1873548a978d27b10644076a0c0 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Sun, 4 Aug 2013 09:13:41 -0700 Subject: [PATCH 412/419] Moved handling of timed out workers within the Master actor --- core/src/main/scala/spark/deploy/DeployMessage.scala | 6 ++++++ core/src/main/scala/spark/deploy/master/Master.scala | 6 +++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 7c37a16615..31861f3ac2 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -109,6 +109,7 @@ private[deploy] object DeployMessages { } // WorkerWebUI to Worker + case object RequestWorkerState // Worker to WorkerWebUI @@ -120,4 +121,9 @@ private[deploy] object DeployMessages { Utils.checkHost(host, "Required hostname") assert (port > 0) } + + // Actor System to Master + + case object CheckForWorkerTimeOut + } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index bd7924c71d..4a4d9908a0 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -80,7 +80,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() - context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) + context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) masterMetricsSystem.registerSource(masterSource) masterMetricsSystem.start() @@ -176,6 +176,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act case RequestMasterState => { sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray) } + + case CheckForWorkerTimeOut => { + timeOutDeadWorkers() + } } /** From bf7033f3ebf9315ccf9aba09a6e702c3a671fd8d Mon Sep 17 00:00:00 2001 From: Ginger Smith Date: Mon, 5 Aug 2013 21:26:24 -0700 Subject: [PATCH 413/419] fixing formatting, style, and input --- .../spark/mllib/util/MFDataGenerator.scala | 73 ++++++++++--------- 1 file changed, 37 insertions(+), 36 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala index 1d2b5c89f0..88992cde0c 100644 --- a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala @@ -28,32 +28,32 @@ import spark.mllib.util.MLUtils * Generate RDD(s) containing data for Matrix Factorization. * * This method samples training entries according to the oversampling factor -* 'tr_samp_fact', which is a multiplicative factor of the number of +* 'trainSampFact', which is a multiplicative factor of the number of * degrees of freedom of the matrix: rank*(m+n-rank). * * It optionally samples entries for a testing matrix using -* 'te_samp_fact', the percentage of the number of training entries +* 'testSampFact', the percentage of the number of training entries * to use for testing. * * This method takes the following inputs: -* sparkMaster (String) The master URL. -* outputPath (String) Directory to save output. -* m (Int) Number of rows in data matrix. -* n (Int) Number of columns in data matrix. -* rank (Int) Underlying rank of data matrix. -* tr_samp_fact (Double) Oversampling factor. -* noise (Boolean) Whether to add gaussian noise to training data. -* sigma (Double) Standard deviation of added gaussian noise. -* test (Boolean) Whether to create testing RDD. -* te_samp_fact (Double) Percentage of training data to use as test data. +* sparkMaster (String) The master URL. +* outputPath (String) Directory to save output. +* m (Int) Number of rows in data matrix. +* n (Int) Number of columns in data matrix. +* rank (Int) Underlying rank of data matrix. +* trainSampFact (Double) Oversampling factor. +* noise (Boolean) Whether to add gaussian noise to training data. +* sigma (Double) Standard deviation of added gaussian noise. +* test (Boolean) Whether to create testing RDD. +* testSampFact (Double) Percentage of training data to use as test data. */ object MFDataGenerator{ def main(args: Array[String]) { - if (args.length != 10) { - println("Usage: MFGenerator " + - " ") + if (args.length < 2) { + println("Usage: MFDataGenerator " + + " [m] [n] [rank] [trainSampFact] [noise] [sigma] [test] [testSampFact]") System.exit(1) } @@ -62,51 +62,52 @@ object MFDataGenerator{ val m: Int = if (args.length > 2) args(2).toInt else 100 val n: Int = if (args.length > 3) args(3).toInt else 100 val rank: Int = if (args.length > 4) args(4).toInt else 10 - val tr_samp_fact: Double = if (args.length > 5) args(5).toDouble else 1.0 + val trainSampFact: Double = if (args.length > 5) args(5).toDouble else 1.0 val noise: Boolean = if (args.length > 6) args(6).toBoolean else false val sigma: Double = if (args.length > 7) args(7).toDouble else 0.1 val test: Boolean = if (args.length > 8) args(8).toBoolean else false - val te_samp_fact: Double = if (args.length > 9) args(9).toDouble else 0.1 + val testSampFact: Double = if (args.length > 9) args(9).toDouble else 0.1 val sc = new SparkContext(sparkMaster, "MFDataGenerator") - val A = DoubleMatrix.randn(m,rank) - val B = DoubleMatrix.randn(rank,n) - val z = 1/(scala.math.sqrt(scala.math.sqrt(rank))) + val A = DoubleMatrix.randn(m, rank) + val B = DoubleMatrix.randn(rank, n) + val z = 1 / (scala.math.sqrt(scala.math.sqrt(rank))) A.mmuli(z) B.mmuli(z) val fullData = A.mmul(B) - val df = rank*(m+n-rank) - val sampsize = scala.math.min(scala.math.round(tr_samp_fact*df), scala.math.round(.99*m*n)).toInt + val df = rank * (m + n - rank) + val sampSize = scala.math.min(scala.math.round(trainSampFact * df), + scala.math.round(.99 * m * n)).toInt val rand = new Random() - val mn = m*n + val mn = m * n val shuffled = rand.shuffle(1 to mn toIterable) - val omega = shuffled.slice(0,sampsize) + val omega = shuffled.slice(0, sampSize) val ordered = omega.sortWith(_ < _).toArray val trainData: RDD[(Int, Int, Double)] = sc.parallelize(ordered) - .map(x => (fullData.indexRows(x-1),fullData.indexColumns(x-1),fullData.get(x-1))) + .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) // optionally add gaussian noise - if(noise){ - trainData.map(x => (x._1,x._2,x._3+rand.nextGaussian*sigma)) + if (noise) { + trainData.map(x => (x._1, x._2, x._3 + rand.nextGaussian * sigma)) } trainData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) // optionally generate testing data - if(test){ - val test_sampsize = scala.math - .min(scala.math.round(sampsize*te_samp_fact),scala.math.round(mn-sampsize)) - .toInt - val test_omega = shuffled.slice(sampsize,sampsize+test_sampsize) - val test_ordered = test_omega.sortWith(_ < _).toArray - val testData: RDD[(Int, Int, Double)] = sc.parallelize(test_ordered) - .map(x=> (fullData.indexRows(x-1),fullData.indexColumns(x-1),fullData.get(x-1))) + if (test) { + val testSampSize = scala.math + .min(scala.math.round(sampSize * testSampFact),scala.math.round(mn - sampSize)).toInt + val testOmega = shuffled.slice(sampSize, sampSize + testSampSize) + val testOrdered = testOmega.sortWith(_ < _).toArray + val testData: RDD[(Int, Int, Double)] = sc.parallelize(testOrdered) + .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) } - sc.stop() + sc.stop() + } } \ No newline at end of file From a30866438bf71c83575a3e83887bd4bf33c7cdde Mon Sep 17 00:00:00 2001 From: Alexander Pivovarov Date: Mon, 5 Aug 2013 21:48:43 -0700 Subject: [PATCH 414/419] SHARK-94 Log the files computed by HadoopRDD and NewHadoopRDD --- core/src/main/scala/spark/rdd/HadoopRDD.scala | 1 + core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index d0fdeb741e..fd00d59c77 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -88,6 +88,7 @@ class HadoopRDD[K, V]( override def compute(theSplit: Partition, context: TaskContext) = new NextIterator[(K, V)] { val split = theSplit.asInstanceOf[HadoopPartition] + logInfo("Input split: " + split.inputSplit) var reader: RecordReader[K, V] = null val conf = confBroadcast.value.value diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 17fe805fd4..0b71608169 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -73,6 +73,7 @@ class NewHadoopRDD[K, V]( override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[NewHadoopPartition] + logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value val attemptId = newTaskAttemptID(jobtrackerId, id, true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) From 42942fc1a917c8ec31f045e7aecfd99bcb0961f6 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 6 Aug 2013 11:59:31 -0700 Subject: [PATCH 415/419] In the process of bringing the GraphLab api back and fixing the analytics toolkit --- .../main/scala/spark/graph/Analytics.scala | 800 +++++++++--------- .../src/main/scala/spark/graph/GraphLab.scala | 4 +- 2 files changed, 403 insertions(+), 401 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 8f76622db0..77d234a5ff 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -1,15 +1,14 @@ package spark.graph import spark._ -import spark.SparkContext._ -// import breeze.linalg._ + object Analytics extends Logging { - def main(args: Array[String]) { - //pregelPagerank() - } +// def main(args: Array[String]) { +// //pregelPagerank() +// } // /** // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD @@ -41,387 +40,394 @@ object Analytics extends Logging { /** * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD */ - def pregelPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { + def pagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + numIter: Int, + resetProb: Double = 0.15) = { // Compute the out degree of each vertex val pagerankGraph = graph.leftJoinVertices[Int, (Int, Double)](graph.outDegrees, (vertex, deg) => (deg.getOrElse(0), 1.0) ) Pregel.iterate[(Int, Double), ED, Double](pagerankGraph)( - (vertex, a: Double) => (vertex.data._1, (0.15 + 0.85 * a)), // apply + (vertex, a: Double) => (vertex.data._1, (resetProb + (1.0 - resetProb) * a)), // apply (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather (a: Double, b: Double) => a + b, // merge 1.0, - numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } + numIter).mapVertices { + case Vertex(id, (outDeg, r)) => Vertex(id, r) + } } -// /** -// * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD -// */ -// def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], -// tol: Float, maxIter: Int = 10) = { -// // Compute the out degree of each vertex -// val pagerankGraph = graph.updateVertices[Int, (Int, Float, Float)](graph.outDegrees, -// (vertex, degIter) => (degIter.sum, 1.0F, 1.0F) -// ) + /** + * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD + */ + def dynamicPagerank[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + tol: Float, + maxIter: Int = Integer.MAX_VALUE, + resetProb: Double = 0.15) = { + // Compute the out degree of each vertex + val pagerankGraph = graph.leftJoinVertices[Int, (Int, Double, Double)](graph.outDegrees, + (vertex, degIter) => (degIter.sum, 1.0, 1.0) + ) -// // Run PageRank -// GraphLab.iterateGAS(pagerankGraph)( -// (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather -// (a: Float, b: Float) => a + b, -// (vertex, a: Option[Float]) => -// (vertex.data._1, (0.15F + 0.85F * a.getOrElse(0F)), vertex.data._2), // apply -// (me_id, edge) => math.abs(edge.src.data._2 - edge.dst.data._1) > tol, // scatter -// maxIter).mapVertices { case Vertex(vid, data) => Vertex(vid, data._2) } -// } - -// /** -// * Compute the connected component membership of each vertex -// * and return an RDD with the vertex value containing the -// * lowest vertex id in the connected component containing -// * that vertex. -// */ -// def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], numIter: Int) = { -// val ccGraph = graph.mapVertices { case Vertex(vid, _) => Vertex(vid, vid) } -// GraphLab.iterateGA[Int, ED, Int](ccGraph)( -// (me_id, edge) => edge.otherVertex(me_id).data, // gather -// (a: Int, b: Int) => math.min(a, b), // merge -// (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply -// numIter, -// gatherDirection = EdgeDirection.Both) -// } - -// /** -// * Compute the shortest path to a set of markers -// */ -// def shortestPath[VD: Manifest](graph: Graph[VD, Float], sources: List[Int], numIter: Int) = { -// val sourceSet = sources.toSet -// val spGraph = graph.mapVertices { -// case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) -// } -// GraphLab.iterateGA[Float, Float, Float](spGraph)( -// (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather -// (a: Float, b: Float) => math.min(a, b), // merge -// (v, a: Option[Float]) => math.min(v.data, a.getOrElse(Float.MaxValue)), // apply -// numIter, -// gatherDirection = EdgeDirection.In) -// } - -// // /** -// // * Compute the connected component membership of each vertex -// // * and return an RDD with the vertex value containing the -// // * lowest vertex id in the connected component containing -// // * that vertex. -// // */ -// // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], -// // numIter: Int = Int.MaxValue) = { - -// // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) -// // val edges = graph.edges // .mapValues(v => None) -// // val ccGraph = new Graph(vertices, edges) - -// // ccGraph.iterateDynamic( -// // (me_id, edge) => edge.otherVertex(me_id).data, // gather -// // (a: Int, b: Int) => math.min(a, b), // merge -// // Integer.MAX_VALUE, -// // (v, a: Int) => math.min(v.data, a), // apply -// // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter -// // numIter, -// // gatherEdges = EdgeDirection.Both, -// // scatterEdges = EdgeDirection.Both).vertices -// // // -// // // graph_ret.vertices.collect.foreach(println) -// // // graph_ret.edges.take(10).foreach(println) -// // } + // Run PageRank + GraphLab.iterate(pagerankGraph)( + (me_id, edge) => edge.src.data._2 / edge.src.data._1, // gather + (a: Double, b: Double) => a + b, + (vertex, a: Option[Double]) => + (vertex.data._1, (resetProb + (1.0 - resetProb) * a.getOrElse(0.0)), vertex.data._2), // apply + (me_id, edge) => math.abs(edge.src.data._3 - edge.src.data._2) > tol, // scatter + maxIter).mapVertices { + case Vertex(vid, data) => Vertex(vid, data._2) + } + } + /** + * Compute the connected component membership of each vertex + * and return an RDD with the vertex value containing the + * lowest vertex id in the connected component containing + * that vertex. + */ + def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]) = { + val ccGraph = graph.mapVertices { case Vertex(vid, _) => vid } + + GraphLab.iterate[Int, ED, Int](ccGraph)( + (me_id, edge) => edge.otherVertex(me_id).data, // gather + (a: Int, b: Int) => math.min(a, b), // merge + (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply + (me_id, edge) => (edge.vertex(me_id).data < edge.otherVertex(me_id).data), // scatter + gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both + ) + } + + // /** + // * Compute the shortest path to a set of markers + // */ + // def shortestPath[VD: Manifest](graph: Graph[VD, Float], sources: List[Int], numIter: Int) = { + // val sourceSet = sources.toSet + // val spGraph = graph.mapVertices { + // case Vertex(vid, _) => Vertex(vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue)) + // } + // GraphLab.iterateGA[Float, Float, Float](spGraph)( + // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather + // (a: Float, b: Float) => math.min(a, b), // merge + // (v, a: Option[Float]) => math.min(v.data, a.getOrElse(Float.MaxValue)), // apply + // numIter, + // gatherDirection = EdgeDirection.In) + // } + + // // /** + // // * Compute the connected component membership of each vertex + // // * and return an RDD with the vertex value containing the + // // * lowest vertex id in the connected component containing + // // * that vertex. + // // */ + // // def dynamicConnectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED], + // // numIter: Int = Int.MaxValue) = { + + // // val vertices = graph.vertices.mapPartitions(iter => iter.map { case (vid, _) => (vid, vid) }) + // // val edges = graph.edges // .mapValues(v => None) + // // val ccGraph = new Graph(vertices, edges) + + // // ccGraph.iterateDynamic( + // // (me_id, edge) => edge.otherVertex(me_id).data, // gather + // // (a: Int, b: Int) => math.min(a, b), // merge + // // Integer.MAX_VALUE, + // // (v, a: Int) => math.min(v.data, a), // apply + // // (me_id, edge) => edge.otherVertex(me_id).data > edge.vertex(me_id).data, // scatter + // // numIter, + // // gatherEdges = EdgeDirection.Both, + // // scatterEdges = EdgeDirection.Both).vertices + // // // + // // // graph_ret.vertices.collect.foreach(println) + // // // graph_ret.edges.take(10).foreach(println) + // // } -// // /** -// // * Compute the shortest path to a set of markers -// // */ -// // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], -// // sources: List[Int], numIter: Int) = { -// // val sourceSet = sources.toSet -// // val vertices = graph.vertices.mapPartitions( -// // iter => iter.map { -// // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) -// // }); + // // /** + // // * Compute the shortest path to a set of markers + // // */ + // // def dynamicShortestPath[VD: Manifest, ED: Manifest](graph: Graph[VD, Float], + // // sources: List[Int], numIter: Int) = { + // // val sourceSet = sources.toSet + // // val vertices = graph.vertices.mapPartitions( + // // iter => iter.map { + // // case (vid, _) => (vid, (if(sourceSet.contains(vid)) 0.0F else Float.MaxValue) ) + // // }); -// // val edges = graph.edges // .mapValues(v => None) -// // val spGraph = new Graph(vertices, edges) + // // val edges = graph.edges // .mapValues(v => None) + // // val spGraph = new Graph(vertices, edges) -// // val niterations = Int.MaxValue -// // spGraph.iterateDynamic( -// // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather -// // (a: Float, b: Float) => math.min(a, b), // merge -// // Float.MaxValue, -// // (v, a: Float) => math.min(v.data, a), // apply -// // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter -// // numIter, -// // gatherEdges = EdgeDirection.In, -// // scatterEdges = EdgeDirection.Out).vertices -// // } + // // val niterations = Int.MaxValue + // // spGraph.iterateDynamic( + // // (me_id, edge) => edge.otherVertex(me_id).data + edge.data, // gather + // // (a: Float, b: Float) => math.min(a, b), // merge + // // Float.MaxValue, + // // (v, a: Float) => math.min(v.data, a), // apply + // // (me_id, edge) => edge.vertex(me_id).data + edge.data < edge.otherVertex(me_id).data, // scatter + // // numIter, + // // gatherEdges = EdgeDirection.In, + // // scatterEdges = EdgeDirection.Out).vertices + // // } + // // /** + // // * + // // */ + // // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], + // // latentK: Int, lambda: Double, numIter: Int) = { + // // val vertices = graph.vertices.mapPartitions( _.map { + // // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) + // // }).cache + // // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // // val edges = graph.edges // .mapValues(v => None) + // // val alsGraph = new Graph(vertices, edges) + // // alsGraph.numVPart = graph.numVPart + // // alsGraph.numEPart = graph.numEPart + + // // val niterations = Int.MaxValue + // // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( + // // (me_id, edge) => { // gather + // // val X = edge.otherVertex(me_id).data + // // val y = edge.data + // // val Xy = X.map(_ * y) + // // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray + // // (Xy, XtX) + // // }, + // // (a, b) => { + // // // The difference between the while loop and the zip is a FACTOR OF TWO in overall + // // // runtime + // // var i = 0 + // // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } + // // i = 0 + // // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } + // // a + // // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) + // // }, + // // (Array.empty[Double], Array.empty[Double]), // default value is empty + // // (vertex, accum) => { // apply + // // val XyArray = accum._1 + // // val XtXArray = accum._2 + // // if(XyArray.isEmpty) vertex.data // no neighbors + // // else { + // // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => + // // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + + // // (if(i == j) lambda else 1.0F) //regularization + // // } + // // val Xy = DenseMatrix.create(latentK,1,XyArray) + // // val w = XtX \ Xy + // // w.data + // // } + // // }, + // // (me_id, edge) => true, + // // numIter, + // // gatherEdges = EdgeDirection.Both, + // // scatterEdges = EdgeDirection.Both, + // // vertex => vertex.id < maxUser).vertices + // // } + + // def main(args: Array[String]) = { + // val host = args(0) + // val taskType = args(1) + // val fname = args(2) + // val options = args.drop(3).map { arg => + // arg.dropWhile(_ == '-').split('=') match { + // case Array(opt, v) => (opt -> v) + // case _ => throw new IllegalArgumentException("Invalid argument: " + arg) + // } + // } + + // System.setProperty("spark.serializer", "spark.KryoSerializer") + // //System.setProperty("spark.shuffle.compress", "false") + // System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") + + // taskType match { + // case "pagerank" => { + + // var numIter = Int.MaxValue + // var isDynamic = false + // var tol:Float = 0.001F + // var outFname = "" + // var numVPart = 4 + // var numEPart = 4 + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case ("tol", v) => tol = v.toFloat + // case ("output", v) => outFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // if(!isDynamic && numIter == Int.MaxValue) { + // println("Set number of iterations!") + // sys.exit(1) + // } + // println("======================================") + // println("| PageRank |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tDynamic: " + isDynamic) + // if(isDynamic) println(" \t |-> Tolerance: " + tol) + // println(" \tNumIter: " + numIter) + // println("======================================") + + // val sc = new SparkContext(host, "PageRank(" + fname + ")") + + // val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() + + // val startTime = System.currentTimeMillis + // logInfo("GRAPHX: starting tasks") + // logInfo("GRAPHX: Number of vertices " + graph.vertices.count) + // logInfo("GRAPHX: Number of edges " + graph.edges.count) + + // val pr = Analytics.pagerank(graph, numIter) + // // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) + // // else Analytics.pagerank(graph, numIter) + // logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) + // if (!outFname.isEmpty) { + // println("Saving pageranks of pages to " + outFname) + // pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) + // } + // logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") + // sc.stop() + // } + + // case "cc" => { + + // var numIter = Int.MaxValue + // var isDynamic = false + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } + + // if(!isDynamic && numIter == Int.MaxValue) { + // println("Set number of iterations!") + // sys.exit(1) + // } + // println("======================================") + // println("| Connected Components |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tDynamic: " + isDynamic) + // println(" \tNumIter: " + numIter) + // println("======================================") + + // val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") + // val graph = Graph.textFile(sc, fname, a => 1.0F) + // val cc = Analytics.connectedComponents(graph, numIter) + // // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) + // // else Analytics.connectedComponents(graph, numIter) + // println("Components: " + cc.vertices.map(_.data).distinct()) + + // sc.stop() + // } + + // case "shortestpath" => { + + // var numIter = Int.MaxValue + // var isDynamic = true + // var sources: List[Int] = List.empty + + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("dynamic", v) => isDynamic = v.toBoolean + // case ("source", v) => sources ++= List(v.toInt) + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } -// // /** -// // * -// // */ -// // def alternatingLeastSquares[VD: ClassManifest, ED: ClassManifest](graph: Graph[VD, Double], -// // latentK: Int, lambda: Double, numIter: Int) = { -// // val vertices = graph.vertices.mapPartitions( _.map { -// // case (vid, _) => (vid, Array.fill(latentK){ scala.util.Random.nextDouble() } ) -// // }).cache -// // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) -// // val edges = graph.edges // .mapValues(v => None) -// // val alsGraph = new Graph(vertices, edges) -// // alsGraph.numVPart = graph.numVPart -// // alsGraph.numEPart = graph.numEPart + // if(!isDynamic && numIter == Int.MaxValue) { + // println("Set number of iterations!") + // sys.exit(1) + // } -// // val niterations = Int.MaxValue -// // alsGraph.iterateDynamic[(Array[Double], Array[Double])]( -// // (me_id, edge) => { // gather -// // val X = edge.otherVertex(me_id).data -// // val y = edge.data -// // val Xy = X.map(_ * y) -// // val XtX = (for(i <- 0 until latentK; j <- i until latentK) yield(X(i) * X(j))).toArray -// // (Xy, XtX) -// // }, -// // (a, b) => { -// // // The difference between the while loop and the zip is a FACTOR OF TWO in overall -// // // runtime -// // var i = 0 -// // while(i < a._1.length) { a._1(i) += b._1(i); i += 1 } -// // i = 0 -// // while(i < a._2.length) { a._2(i) += b._2(i); i += 1 } -// // a -// // // (a._1.zip(b._1).map{ case (q,r) => q+r }, a._2.zip(b._2).map{ case (q,r) => q+r }) -// // }, -// // (Array.empty[Double], Array.empty[Double]), // default value is empty -// // (vertex, accum) => { // apply -// // val XyArray = accum._1 -// // val XtXArray = accum._2 -// // if(XyArray.isEmpty) vertex.data // no neighbors -// // else { -// // val XtX = DenseMatrix.tabulate(latentK,latentK){ (i,j) => -// // (if(i < j) XtXArray(i + (j+1)*j/2) else XtXArray(i + (j+1)*j/2)) + -// // (if(i == j) lambda else 1.0F) //regularization -// // } -// // val Xy = DenseMatrix.create(latentK,1,XyArray) -// // val w = XtX \ Xy -// // w.data -// // } -// // }, -// // (me_id, edge) => true, -// // numIter, -// // gatherEdges = EdgeDirection.Both, -// // scatterEdges = EdgeDirection.Both, -// // vertex => vertex.id < maxUser).vertices -// // } + // if(sources.isEmpty) { + // println("No sources provided!") + // sys.exit(1) + // } -// def main(args: Array[String]) = { -// val host = args(0) -// val taskType = args(1) -// val fname = args(2) -// val options = args.drop(3).map { arg => -// arg.dropWhile(_ == '-').split('=') match { -// case Array(opt, v) => (opt -> v) -// case _ => throw new IllegalArgumentException("Invalid argument: " + arg) -// } -// } + // println("======================================") + // println("| Shortest Path |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tDynamic: " + isDynamic) + // println(" \tNumIter: " + numIter) + // println(" \tSources: [" + sources.mkString(", ") + "]") + // println("======================================") -// System.setProperty("spark.serializer", "spark.KryoSerializer") -// //System.setProperty("spark.shuffle.compress", "false") -// System.setProperty("spark.kryo.registrator", "spark.graph.GraphKryoRegistrator") + // val sc = new SparkContext(host, "ShortestPath(" + fname + ")") + // val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) + // val sp = Analytics.shortestPath(graph, sources, numIter) + // // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) + // // else Analytics.shortestPath(graph, sources, numIter) + // println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) -// taskType match { -// case "pagerank" => { - -// var numIter = Int.MaxValue -// var isDynamic = false -// var tol:Float = 0.001F -// var outFname = "" -// var numVPart = 4 -// var numEPart = 4 - -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case ("tol", v) => tol = v.toFloat -// case ("output", v) => outFname = v -// case ("numVPart", v) => numVPart = v.toInt -// case ("numEPart", v) => numEPart = v.toInt -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } - -// if(!isDynamic && numIter == Int.MaxValue) { -// println("Set number of iterations!") -// sys.exit(1) -// } -// println("======================================") -// println("| PageRank |") -// println("--------------------------------------") -// println(" Using parameters:") -// println(" \tDynamic: " + isDynamic) -// if(isDynamic) println(" \t |-> Tolerance: " + tol) -// println(" \tNumIter: " + numIter) -// println("======================================") - -// val sc = new SparkContext(host, "PageRank(" + fname + ")") - -// val graph = Graph.textFile(sc, fname, a => 1.0F).withPartitioner(numVPart, numEPart).cache() - -// val startTime = System.currentTimeMillis -// logInfo("GRAPHX: starting tasks") -// logInfo("GRAPHX: Number of vertices " + graph.vertices.count) -// logInfo("GRAPHX: Number of edges " + graph.edges.count) - -// val pr = Analytics.pagerank(graph, numIter) -// // val pr = if(isDynamic) Analytics.dynamicPagerank(graph, tol, numIter) -// // else Analytics.pagerank(graph, numIter) -// logInfo("GRAPHX: Total rank: " + pr.vertices.map{ case Vertex(id,r) => r }.reduce(_+_) ) -// if (!outFname.isEmpty) { -// println("Saving pageranks of pages to " + outFname) -// pr.vertices.map{case Vertex(id, r) => id + "\t" + r}.saveAsTextFile(outFname) -// } -// logInfo("GRAPHX: Runtime: " + ((System.currentTimeMillis - startTime)/1000.0) + " seconds") -// sc.stop() -// } - -// case "cc" => { - -// var numIter = Int.MaxValue -// var isDynamic = false - -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } - -// if(!isDynamic && numIter == Int.MaxValue) { -// println("Set number of iterations!") -// sys.exit(1) -// } -// println("======================================") -// println("| Connected Components |") -// println("--------------------------------------") -// println(" Using parameters:") -// println(" \tDynamic: " + isDynamic) -// println(" \tNumIter: " + numIter) -// println("======================================") - -// val sc = new SparkContext(host, "ConnectedComponents(" + fname + ")") -// val graph = Graph.textFile(sc, fname, a => 1.0F) -// val cc = Analytics.connectedComponents(graph, numIter) -// // val cc = if(isDynamic) Analytics.dynamicConnectedComponents(graph, numIter) -// // else Analytics.connectedComponents(graph, numIter) -// println("Components: " + cc.vertices.map(_.data).distinct()) - -// sc.stop() -// } - -// case "shortestpath" => { - -// var numIter = Int.MaxValue -// var isDynamic = true -// var sources: List[Int] = List.empty - -// options.foreach{ -// case ("numIter", v) => numIter = v.toInt -// case ("dynamic", v) => isDynamic = v.toBoolean -// case ("source", v) => sources ++= List(v.toInt) -// case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// } + // sc.stop() + // } -// if(!isDynamic && numIter == Int.MaxValue) { -// println("Set number of iterations!") -// sys.exit(1) -// } + // // case "als" => { -// if(sources.isEmpty) { -// println("No sources provided!") -// sys.exit(1) -// } + // // var numIter = 5 + // // var lambda = 0.01 + // // var latentK = 10 + // // var usersFname = "usersFactors.tsv" + // // var moviesFname = "moviesFname.tsv" + // // var numVPart = 4 + // // var numEPart = 4 -// println("======================================") -// println("| Shortest Path |") -// println("--------------------------------------") -// println(" Using parameters:") -// println(" \tDynamic: " + isDynamic) -// println(" \tNumIter: " + numIter) -// println(" \tSources: [" + sources.mkString(", ") + "]") -// println("======================================") + // // options.foreach{ + // // case ("numIter", v) => numIter = v.toInt + // // case ("lambda", v) => lambda = v.toDouble + // // case ("latentK", v) => latentK = v.toInt + // // case ("usersFname", v) => usersFname = v + // // case ("moviesFname", v) => moviesFname = v + // // case ("numVPart", v) => numVPart = v.toInt + // // case ("numEPart", v) => numEPart = v.toInt + // // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // // } -// val sc = new SparkContext(host, "ShortestPath(" + fname + ")") -// val graph = Graph.textFile(sc, fname, a => (if(a.isEmpty) 1.0F else a(0).toFloat ) ) -// val sp = Analytics.shortestPath(graph, sources, numIter) -// // val cc = if(isDynamic) Analytics.dynamicShortestPath(graph, sources, numIter) -// // else Analytics.shortestPath(graph, sources, numIter) -// println("Longest Path: " + sp.vertices.map(_.data).reduce(math.max(_,_))) + // // println("======================================") + // // println("| Alternating Least Squares |") + // // println("--------------------------------------") + // // println(" Using parameters:") + // // println(" \tNumIter: " + numIter) + // // println(" \tLambda: " + lambda) + // // println(" \tLatentK: " + latentK) + // // println(" \tusersFname: " + usersFname) + // // println(" \tmoviesFname: " + moviesFname) + // // println("======================================") -// sc.stop() -// } + // // val sc = new SparkContext(host, "ALS(" + fname + ")") + // // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) + // // graph.numVPart = numVPart + // // graph.numEPart = numEPart + + // // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) + // // assert(maxUser < minMovie) + + // // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache + // // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) + // // .saveAsTextFile(usersFname) + // // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) + // // .saveAsTextFile(moviesFname) + + // // sc.stop() + // // } -// // case "als" => { - -// // var numIter = 5 -// // var lambda = 0.01 -// // var latentK = 10 -// // var usersFname = "usersFactors.tsv" -// // var moviesFname = "moviesFname.tsv" -// // var numVPart = 4 -// // var numEPart = 4 - -// // options.foreach{ -// // case ("numIter", v) => numIter = v.toInt -// // case ("lambda", v) => lambda = v.toDouble -// // case ("latentK", v) => latentK = v.toInt -// // case ("usersFname", v) => usersFname = v -// // case ("moviesFname", v) => moviesFname = v -// // case ("numVPart", v) => numVPart = v.toInt -// // case ("numEPart", v) => numEPart = v.toInt -// // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) -// // } - -// // println("======================================") -// // println("| Alternating Least Squares |") -// // println("--------------------------------------") -// // println(" Using parameters:") -// // println(" \tNumIter: " + numIter) -// // println(" \tLambda: " + lambda) -// // println(" \tLatentK: " + latentK) -// // println(" \tusersFname: " + usersFname) -// // println(" \tmoviesFname: " + moviesFname) -// // println("======================================") - -// // val sc = new SparkContext(host, "ALS(" + fname + ")") -// // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) -// // graph.numVPart = numVPart -// // graph.numEPart = numEPart - -// // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) -// // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) -// // assert(maxUser < minMovie) - -// // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache -// // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) -// // .saveAsTextFile(usersFname) -// // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) -// // .saveAsTextFile(moviesFname) - -// // sc.stop() -// // } - - -// case _ => { -// println("Invalid task type.") -// } -// } -// } + // case _ => { + // println("Invalid task type.") + // } + // } + // } // /** // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD @@ -503,8 +509,6 @@ object Analytics extends Logging { // } - - // /** // * Compute the shortest path to a set of markers // */ @@ -532,8 +536,6 @@ object Analytics extends Logging { // } - - // /** // * // */ @@ -735,55 +737,55 @@ object Analytics extends Logging { // } - // case "als" => { + // case "als" => { - // var numIter = 5 - // var lambda = 0.01 - // var latentK = 10 - // var usersFname = "usersFactors.tsv" - // var moviesFname = "moviesFname.tsv" - // var numVPart = 4 - // var numEPart = 4 + // var numIter = 5 + // var lambda = 0.01 + // var latentK = 10 + // var usersFname = "usersFactors.tsv" + // var moviesFname = "moviesFname.tsv" + // var numVPart = 4 + // var numEPart = 4 - // options.foreach{ - // case ("numIter", v) => numIter = v.toInt - // case ("lambda", v) => lambda = v.toDouble - // case ("latentK", v) => latentK = v.toInt - // case ("usersFname", v) => usersFname = v - // case ("moviesFname", v) => moviesFname = v - // case ("numVPart", v) => numVPart = v.toInt - // case ("numEPart", v) => numEPart = v.toInt - // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) - // } + // options.foreach{ + // case ("numIter", v) => numIter = v.toInt + // case ("lambda", v) => lambda = v.toDouble + // case ("latentK", v) => latentK = v.toInt + // case ("usersFname", v) => usersFname = v + // case ("moviesFname", v) => moviesFname = v + // case ("numVPart", v) => numVPart = v.toInt + // case ("numEPart", v) => numEPart = v.toInt + // case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) + // } - // println("======================================") - // println("| Alternating Least Squares |") - // println("--------------------------------------") - // println(" Using parameters:") - // println(" \tNumIter: " + numIter) - // println(" \tLambda: " + lambda) - // println(" \tLatentK: " + latentK) - // println(" \tusersFname: " + usersFname) - // println(" \tmoviesFname: " + moviesFname) - // println("======================================") + // println("======================================") + // println("| Alternating Least Squares |") + // println("--------------------------------------") + // println(" Using parameters:") + // println(" \tNumIter: " + numIter) + // println(" \tLambda: " + lambda) + // println(" \tLatentK: " + latentK) + // println(" \tusersFname: " + usersFname) + // println(" \tmoviesFname: " + moviesFname) + // println("======================================") - // val sc = new SparkContext(host, "ALS(" + fname + ")") - // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) - // graph.numVPart = numVPart - // graph.numEPart = numEPart + // val sc = new SparkContext(host, "ALS(" + fname + ")") + // val graph = Graph.textFile(sc, fname, a => a(0).toDouble ) + // graph.numVPart = numVPart + // graph.numEPart = numEPart - // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) - // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) - // assert(maxUser < minMovie) + // val maxUser = graph.edges.map(_._1).reduce(math.max(_,_)) + // val minMovie = graph.edges.map(_._2).reduce(math.min(_,_)) + // assert(maxUser < minMovie) - // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache - // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(usersFname) - // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) - // .saveAsTextFile(moviesFname) + // val factors = Analytics.alternatingLeastSquares(graph, latentK, lambda, numIter).cache + // factors.filter(_._1 <= maxUser).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(usersFname) + // factors.filter(_._1 >= minMovie).map(r => r._1 + "\t" + r._2.mkString("\t")) + // .saveAsTextFile(moviesFname) - // sc.stop() - // } + // sc.stop() + // } // case _ => { diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 504cd162ab..f89c2a39d7 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -33,12 +33,12 @@ object GraphLab { * @tparam A The type accumulated during the gather phase * @return the resulting graph after the algorithm converges */ - def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, mergeFunc: (A, A) => A, applyFunc: (Vertex[VD], Option[A]) => VD, scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean, - numIter: Int, + numIter: Int = Integer.MAX_VALUE, gatherDirection: EdgeDirection = EdgeDirection.In, scatterDirection: EdgeDirection = EdgeDirection.Out): Graph[VD, ED] = { From 7ae83f6ef4200eb992aa764fc6be987f55e1ba47 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 6 Aug 2013 14:05:54 -0700 Subject: [PATCH 416/419] Switching to Long vids instead of integers. This required a surprising number of changes since the fastutil library function names include the type (e.g., getLong() instead of just get()) --- .../src/main/scala/spark/graph/Analytics.scala | 6 +++--- .../scala/spark/graph/impl/EdgePartition.scala | 4 ++-- .../spark/graph/impl/EdgeTripletRDD.scala | 4 ++-- .../scala/spark/graph/impl/GraphImpl.scala | 18 +++++++++--------- graph/src/main/scala/spark/graph/package.scala | 6 ++++-- 5 files changed, 20 insertions(+), 18 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index df4db60339..601a0785e1 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -87,10 +87,10 @@ object Analytics extends Logging { def connectedComponents[VD: Manifest, ED: Manifest](graph: Graph[VD, ED]) = { val ccGraph = graph.mapVertices { case Vertex(vid, _) => vid } - GraphLab.iterate[Int, ED, Int](ccGraph)( + GraphLab.iterate(ccGraph)( (me_id, edge) => edge.otherVertex(me_id).data, // gather - (a: Int, b: Int) => math.min(a, b), // merge - (v, a: Option[Int]) => math.min(v.data, a.getOrElse(Integer.MAX_VALUE)), // apply + (a: Vid, b: Vid) => math.min(a, b), // merge + (v, a: Option[Vid]) => math.min(v.data, a.getOrElse(Long.MaxValue)), // apply (me_id, edge) => (edge.vertex(me_id).data < edge.otherVertex(me_id).data), // scatter gatherDirection = EdgeDirection.Both, scatterDirection = EdgeDirection.Both ) diff --git a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala index 0e092541c9..4e0d5f41b9 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala @@ -16,8 +16,8 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) private var _data: Array[ED] = _ private var _dataBuilder = ArrayBuilder.make[ED] - val srcIds: IntArrayList = new IntArrayList - val dstIds: IntArrayList = new IntArrayList + val srcIds = new VertexArrayList + val dstIds = new VertexArrayList def data: Array[ED] = _data diff --git a/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala b/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala index f157dd4056..f6de8e59af 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala @@ -63,11 +63,11 @@ class EdgeTripletRDD[VD: ClassManifest, ED: ClassManifest]( override def hasNext: Boolean = pos < edgePartition.size override def next() = { - e.src.id = edgePartition.srcIds.getInt(pos) + e.src.id = edgePartition.srcIds.getLong(pos) // assert(vmap.containsKey(e.src.id)) e.src.data = vmap.get(e.src.id) - e.dst.id = edgePartition.dstIds.getInt(pos) + e.dst.id = edgePartition.dstIds.getLong(pos) // assert(vmap.containsKey(e.dst.id)) e.dst.data = vmap.get(e.dst.id) diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 775d0686e7..024e3e8d02 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -147,8 +147,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } } - vmap.int2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => - (entry.getIntKey(), entry.getValue()._2) + vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => + (entry.getLongKey(), entry.getValue()._2) } } .map{ case (vid, aOpt) => (vid, aOpt.get) } @@ -202,8 +202,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } } - vmap.int2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => - (entry.getIntKey(), entry.getValue()._2) + vmap.long2ObjectEntrySet().fastIterator().filter(!_.getValue()._2.isEmpty).map{ entry => + (entry.getLongKey(), entry.getValue()._2) } } .map{ case (vid, aOpt) => (vid, aOpt.get) } @@ -327,16 +327,16 @@ object GraphImpl { // A key-value RDD. The key is a vertex id, and the value is a list of // partitions that contains edges referencing the vertex. - val vid2pid : RDD[(Int, Seq[Pid])] = eTable.mapPartitions { iter => + val vid2pid : RDD[(Vid, Seq[Pid])] = eTable.mapPartitions { iter => val (pid, edgePartition) = iter.next() - val vSet = new it.unimi.dsi.fastutil.ints.IntOpenHashSet + val vSet = new VertexSet var i = 0 while (i < edgePartition.srcIds.size) { - vSet.add(edgePartition.srcIds.getInt(i)) - vSet.add(edgePartition.dstIds.getInt(i)) + vSet.add(edgePartition.srcIds.getLong(i)) + vSet.add(edgePartition.dstIds.getLong(i)) i += 1 } - vSet.iterator.map { vid => (vid.intValue, pid) } + vSet.iterator.map { vid => (vid.toLong, pid) } }.groupByKey(partitioner) vertices diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala index cf1b23ca5d..d95dcdce08 100644 --- a/graph/src/main/scala/spark/graph/package.scala +++ b/graph/src/main/scala/spark/graph/package.scala @@ -2,10 +2,12 @@ package spark package object graph { - type Vid = Int + type Vid = Long type Pid = Int - type VertexHashMap[T] = it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap[T] + type VertexHashMap[T] = it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap[T] + type VertexSet = it.unimi.dsi.fastutil.longs.LongOpenHashSet + type VertexArrayList = it.unimi.dsi.fastutil.longs.LongArrayList /** * Return the default null-like value for a data type T. From b454314e070a70e51b41c76d6a54bdd038d22418 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 6 Aug 2013 15:14:13 -0700 Subject: [PATCH 417/419] Added 2d partitioning --- .../scala/spark/graph/impl/GraphImpl.scala | 76 ++++++++++++++++++- 1 file changed, 74 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 024e3e8d02..cbd1f27afb 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -2,13 +2,16 @@ package spark.graph.impl import scala.collection.JavaConversions._ -import spark.{ClosureCleaner, HashPartitioner, RDD} +import spark.{ClosureCleaner, Partitioner, HashPartitioner, RDD} import spark.SparkContext._ import spark.graph._ import spark.graph.impl.GraphImpl._ + + + /** * A Graph RDD that supports computation on graphs. */ @@ -294,6 +297,67 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( object GraphImpl { + + protected def edgePartitionFunction1D(src: Vid, dst: Vid, numParts: Pid): Pid = { + val mixingPrime: Vid = 1125899906842597L + (math.abs(src) * mixingPrime).toInt % numParts + } + + + + /** + * This function implements a classic 2D-Partitioning of a sparse matrix. + * Suppose we have a graph with 11 vertices that we want to partition + * over 9 machines. We can use the following sparse matrix representation: + * + * __________________________________ + * v0 | P0 * | P1 | P2 * | + * v1 | **** | * | | + * v2 | ******* | ** | **** | + * v3 | ***** | * * | * | + * ---------------------------------- + * v4 | P3 * | P4 *** | P5 ** * | + * v5 | * * | * | | + * v6 | * | ** | **** | + * v7 | * * * | * * | * | + * ---------------------------------- + * v8 | P6 * | P7 * | P8 * *| + * v9 | * | * * | | + * v10 | * | ** | * * | + * v11 | * <-E | *** | ** | + * ---------------------------------- + * + * The edge denoted by E connects v11 with v1 and is assigned to + * processor P6. To get the processor number we divide the matrix + * into sqrt(numProc) by sqrt(numProc) blocks. Notice that edges + * adjacent to v11 can only be in the first colum of + * blocks (P0, P3, P6) or the last row of blocks (P6, P7, P8). + * As a consequence we can guarantee that v11 will need to be + * replicated to at most 2 * sqrt(numProc) machines. + * + * Notice that P0 has many edges and as a consequence this + * partitioning would lead to poor work balance. To improve + * balance we first multiply each vertex id by a large prime + * to effectively suffle the vertex locations. + * + * One of the limitations of this approach is that the number of + * machines must either be a perfect square. We partially address + * this limitation by computing the machine assignment to the next + * largest perfect square and then mapping back down to the actual + * number of machines. Unfortunately, this can also lead to work + * imbalance and so it is suggested that a perfect square is used. + * + * + */ + protected def edgePartitionFunction2D(src: Vid, dst: Vid, + numParts: Pid, ceilSqrtNumParts: Pid): Pid = { + val mixingPrime: Vid = 1125899906842597L + val col: Pid = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt + val row: Pid = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + (col * ceilSqrtNumParts + row) % numParts + } + + /** * Create the edge table RDD, which is much more efficient for Java heap storage than the * normal edges data structure (RDD[(Vid, Vid, ED)]). @@ -304,10 +368,18 @@ object GraphImpl { */ protected def createETable[ED: ClassManifest](edges: RDD[Edge[ED]], numPartitions: Int) : RDD[(Pid, EdgePartition[ED])] = { + val ceilSqrt: Pid = math.ceil(math.sqrt(numPartitions)).toInt + edges .map { e => // Random partitioning based on the source vertex id. - (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) + // val part: Pid = edgePartitionFunction1D(e.src, e.dst, numPartitions) + val part: Pid = edgePartitionFunction2D(e.src, e.dst, numPartitions, ceilSqrt) + + // Should we be using 3-tuple or an optimized class + (part, (e.src, e.dst, e.data)) + // (math.abs(e.src) % numPartitions, (e.src, e.dst, e.data)) + } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex({ (pid, iter) => From ddf126edad11f0a3e0c854beb32ec9543e98bd06 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 6 Aug 2013 17:48:04 -0700 Subject: [PATCH 418/419] added subgraph --- graph/src/main/scala/spark/graph/Graph.scala | 29 +++++++++++++++++++ .../scala/spark/graph/impl/GraphImpl.scala | 17 +++++++++-- .../test/scala/spark/graph/GraphSuite.scala | 3 +- 3 files changed, 46 insertions(+), 3 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index f51171f0d6..095ac2f7d9 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -141,6 +141,13 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapTriplets[ED2: ClassManifest]( map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] + + /** + * Remove edges conntecting vertices that are not in the graph. + * + * @todo remove this function and ensure that for a graph G=(V,E): + * if (u,v) in E then u in V and v in V + */ def correctEdges(): Graph[VD, ED] /** @@ -150,6 +157,28 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { */ def reverse: Graph[VD, ED] + + /** + * This function takes a vertex and edge predicate and constructs the subgraph + * that consists of vertices and edges that satisfy the predict. The resulting + * graph contains the vertices and edges that satisfy: + * + * V' = {v : for all v in V where vpred(v)} + * E' = {(u,v): for all (u,v) in E where epred((u,v)) && vpred(u) && vpred(v)} + * + * @param epred the edge predicate which takes a triplet and evaluates to true + * if the edge is to remain in the subgraph. Note that only edges in which both + * vertices satisfy the vertex predicate are considered. + * + * @param vpred the vertex predicate which takes a vertex object and evaluates + * to true if the vertex is to be included in the subgraph + * + * @return the subgraph containing only the vertices and edges that satisfy the + * predicates. + */ + def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (_ => true), + vpred: Vertex[VD] => Boolean = (_ => true) ): Graph[VD, ED] + /** * This function is used to compute a statistic for the neighborhood of each * vertex. diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index cbd1f27afb..08fc016a43 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -101,6 +101,19 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( Graph(vertices, newEdges) } + + override def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (_ => true), + vpred: Vertex[VD] => Boolean = (_ => true) ): Graph[VD, ED] = { + + // Restrict the set of vertices to those that satisfy the vertex predicate + val newVertices = vertices.filter(vpred) + // Restrict the set of edges to those that satisfy the vertex and the edge predicate. + val newEdges = triplets.filter(t => vpred(t.src) && vpred(t.dst) && epred(t)) + .map( t => Edge(t.src.id, t.dst.id, t.data) ) + + new GraphImpl(newVertices, newEdges) + } + ////////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods ////////////////////////////////////////////////////////////////////////////////////////////////// @@ -303,7 +316,7 @@ object GraphImpl { (math.abs(src) * mixingPrime).toInt % numParts } - + /** * This function implements a classic 2D-Partitioning of a sparse matrix. @@ -340,7 +353,7 @@ object GraphImpl { * balance we first multiply each vertex id by a large prime * to effectively suffle the vertex locations. * - * One of the limitations of this approach is that the number of + * One of the limitations of this approach is that the number of * machines must either be a perfect square. We partially address * this limitation by computing the machine assignment to the next * largest perfect square and then mapping back down to the actual diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index 87c8c158af..ba438a7207 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -12,7 +12,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { } - test("joinVertices") { + /* test("joinVertices") { sc = new SparkContext("local", "test") val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"), Vertex(3, "three")), 2) val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) @@ -29,6 +29,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val e = g1.edges.collect() assert(e(0).data === "onetwo") } + */ // test("graph partitioner") { // sc = new SparkContext("local", "test") From 5ccb60d467f58c104f37e05e99a50fdf06301e5e Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 11 Aug 2013 14:49:22 -0700 Subject: [PATCH 419/419] Working on graph test suite --- graph/src/main/scala/spark/graph/Graph.scala | 11 ++++++++++- graph/src/test/scala/spark/graph/GraphSuite.scala | 12 +++++++++++- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 095ac2f7d9..594b3b5495 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -179,6 +179,15 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def subgraph(epred: EdgeTriplet[VD,ED] => Boolean = (_ => true), vpred: Vertex[VD] => Boolean = (_ => true) ): Graph[VD, ED] + + // /** + // * Combine the attrributes of edges connecting the same vertices. + // * + // * @todo Do we want to support this function + // */ + // def combineEdges(reduce: (ED, ED) => ED): Graph[VD, ED] + + /** * This function is used to compute a statistic for the neighborhood of each * vertex. @@ -360,7 +369,7 @@ object Graph { import spark.graph.impl._ import spark.SparkContext._ - def apply(rawEdges: RDD[(Int, Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = { + def apply(rawEdges: RDD[(Vid, Vid)], uniqueEdges: Boolean = true): Graph[Int, Int] = { // Reduce to unique edges. val edges: RDD[Edge[Int]] = if (uniqueEdges) { diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index ba438a7207..804542f1bd 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -3,11 +3,21 @@ package spark.graph import org.scalatest.FunSuite import spark.SparkContext -import spark.graph.impl.GraphImpl +import spark.graph._ + class GraphSuite extends FunSuite with LocalSparkContext { +// val sc = new SparkContext("local[4]", "test") + + test("Graph Creation") { + val rawEdges = (0L to 100L).zip((1L to 99L) :+ 0L) + val edges = sc.parallelize(rawEdges) + val graph = Graph(edges) + assert( graph.edges.count() === rawEdges.size ) + } + test("aggregateNeighbors") { }
    Stage IdPool NamePool NameOrigin Submitted Duration
    {s.id}{if (stagePoolInfo.hasHref) { - {poolName} - } else { - {poolName} - }}{poolName.get}{s.name} {submissionTime} {getElapsedTime(s.submissionTime, From 37bc64a20535215bd4638a928431ebd0e45f3a1a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Jul 2013 11:57:44 -0700 Subject: [PATCH 396/419] Adding application-level metrics. This adds metrics for applications in the deploy Master. --- .../spark/deploy/master/ApplicationInfo.scala | 1 + .../deploy/master/ApplicationSource.scala | 24 +++++++++++++++++++ .../scala/spark/deploy/master/Master.scala | 22 +++++++++++++---- .../scala/spark/metrics/MetricsSystem.scala | 10 +++++++- 4 files changed, 51 insertions(+), 6 deletions(-) create mode 100644 core/src/main/scala/spark/deploy/master/ApplicationSource.scala diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 15ff919738..79687df614 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -34,6 +34,7 @@ private[spark] class ApplicationInfo( var executors = new mutable.HashMap[Int, ExecutorInfo] var coresGranted = 0 var endTime = -1L + val appSource = new ApplicationSource(this) private var nextExecutorId = 0 diff --git a/core/src/main/scala/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/spark/deploy/master/ApplicationSource.scala new file mode 100644 index 0000000000..4df2b6bfdd --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/ApplicationSource.scala @@ -0,0 +1,24 @@ +package spark.deploy.master + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import spark.metrics.source.Source + +class ApplicationSource(val application: ApplicationInfo) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "%s.%s.%s".format("application", application.desc.name, + System.currentTimeMillis()) + + metricRegistry.register(MetricRegistry.name("status"), new Gauge[String] { + override def getValue: String = application.state.toString + }) + + metricRegistry.register(MetricRegistry.name("runtime_ms"), new Gauge[Long] { + override def getValue: Long = application.duration + }) + + metricRegistry.register(MetricRegistry.name("cores", "number"), new Gauge[Int] { + override def getValue: Int = application.coresGranted + }) + +} diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 202d5bcdb7..36c22d0873 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -38,6 +38,7 @@ import spark.util.AkkaUtils private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 + val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retained_applications", "1000").toInt var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] @@ -59,7 +60,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val metricsSystem = MetricsSystem.createMetricsSystem("master") + val masterMetricsSystem = MetricsSystem.createMetricsSystem("master") + val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications") val masterSource = new MasterSource(this) val masterPublicAddress = { @@ -79,13 +81,15 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - metricsSystem.registerSource(masterSource) - metricsSystem.start() + masterMetricsSystem.registerSource(masterSource) + masterMetricsSystem.start() + applicationMetricsSystem.start() } override def postStop() { webUi.stop() - metricsSystem.stop() + masterMetricsSystem.stop() + applicationMetricsSystem.stop() } override def receive = { @@ -275,6 +279,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val now = System.currentTimeMillis() val date = new Date(now) val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl) + applicationMetricsSystem.registerSource(app.appSource) apps += app idToApp(app.id) = app actorToApp(driver) = app @@ -300,7 +305,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act idToApp -= app.id actorToApp -= app.driver addressToApp -= app.driver.path.address - completedApps += app // Remember it in our history + if (completedApps.size >= RETAINED_APPLICATIONS) { + val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) + completedApps.take(toRemove).foreach( a => { + applicationMetricsSystem.removeSource(a.appSource) + }) + completedApps.trimStart(toRemove) + } + completedApps += app // Remember it in our history waitingApps -= app for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index fabddfb947..5c98f892e1 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -17,7 +17,7 @@ package spark.metrics -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} +import com.codahale.metrics._ import java.util.Properties import java.util.concurrent.TimeUnit @@ -93,6 +93,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } } + def removeSource(source: Source) { + sources -= source + println("Removing source: " + source.sourceName) + registry.removeMatching(new MetricFilter { + def matches(name: String, metric: Metric): Boolean = name.startsWith(source.sourceName) + }) + } + def registerSources() { val instConfig = metricsConfig.getInstance(instance) val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) From 12d9c82c9b2a5040a2d1e5ab23ab44b41cbe1807 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Jul 2013 13:17:23 -0700 Subject: [PATCH 397/419] Small style fix --- core/src/main/scala/spark/metrics/MetricsSystem.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 5c98f892e1..1dacafa135 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -17,7 +17,7 @@ package spark.metrics -import com.codahale.metrics._ +import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit @@ -95,7 +95,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def removeSource(source: Source) { sources -= source - println("Removing source: " + source.sourceName) registry.removeMatching(new MetricFilter { def matches(name: String, metric: Metric): Boolean = name.startsWith(source.sourceName) }) From f1d2ad550edb83319fe3e187dad2151a8d0aeae1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Jul 2013 21:40:56 -0700 Subject: [PATCH 398/419] under_scores --> camelCase for config options --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 36c22d0873..0aed4b9802 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -38,7 +38,7 @@ import spark.util.AkkaUtils private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 - val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retained_applications", "1000").toInt + val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt var nextAppNumber = 0 val workers = new HashSet[WorkerInfo] From d3c37ff120f42f4291a5970fb842af79e0f4a866 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 29 Jul 2013 22:09:22 -0700 Subject: [PATCH 399/419] Improving documentation in config file example --- conf/metrics.properties.template | 67 +++++++++++++++----------------- 1 file changed, 32 insertions(+), 35 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index 0486ca4c79..63a5a2093e 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,48 +1,45 @@ -# syntax: [instance].[sink|source].[name].[options] +# syntax: [instance].sink|source.[name].[options]=[value] -# "instance" specify "who" (the role) use metrics system. In spark there are -# several roles like master, worker, executor, driver, these roles will -# create metrics system for monitoring. So instance represents these roles. -# Currently in Spark, several instances have already implemented: master, -# worker, executor, driver. +# This file configures Spark's internal metrics system. The metrics system is +# divided into instances which correspond to internal components. +# Each instance can be configured to report its metrics to one or more sinks. +# Accepted values for [instance] are "master", "worker", "executor", "driver", +# and "applications". A wild card "*" can be used as an instance name, in +# which case all instances will inherit the supplied property. # -# [instance] field can be "master", "worker", "executor", "driver", which means -# only the specified instance has this property. -# a wild card "*" can be used to represent instance name, which means all the -# instances will have this property. +# Within an instance, a "source" specifies a particular set of grouped metrics. +# there are two kinds of sources: +# 1. Spark internal sources, like MasterSource, WorkerSource, etc, which will +# collect a Spark component's internal state. Each instance is paired with a +# Spark source that is added automatically. +# 2. Common sources, like JvmSource, which will collect low level state. +# These can be added through configuration options and are then loaded +# using reflection. # -# "source" specify "where" (source) to collect metrics data. In metrics system, -# there exists two kinds of source: -# 1. Spark internal source, like MasterSource, WorkerSource, etc, which will -# collect Spark component's internal state, these sources are related to -# instance and will be added after specific metrics system is created. -# 2. Common source, like JvmSource, which will collect low level state, is -# configured by configuration and loaded through reflection. +# A "sink" specifies where metrics are delivered to. Each instance can be +# assigned one or more sinks. # -# "sink" specify "where" (destination) to output metrics data to. Several sinks -# can be coexisted and flush metrics to all these sinks. +# The sink|source field specifies whether the property relates to a sink or +# source. # -# [sink|source] field specify this property is source related or sink, this -# field can only be source or sink. +# The [name] field specifies the name of source or sink. # -# [name] field specify the name of source or sink, this is custom defined. -# -# [options] field is the specific property of this source or sink, this source -# or sink is responsible for parsing this property. +# The [options] field is the specific property of this source or sink. The +# source or sink is responsible for parsing this property. # # Notes: -# 1. Sinks should be added through configuration, like console sink, class -# full name should be specified by class property. -# 2. Some sinks can specify polling period, like console sink, which is 10 seconds, -# it should be attention minimal polling period is 1 seconds, any period -# below than 1s is illegal. -# 3. Wild card property can be overlapped by specific instance property, for -# example, *.sink.console.period can be overlapped by master.sink.console.period. +# 1. To add a new sink, set the "class" option to a fully qualified class +# name (see examples below). +# 2. Some sinks involve a polling period. The minimum allowed polling period +# is 1 second. +# 3. Wild card properties can be overridden by more specific properties. +# For example, master.sink.console.period takes precedence over +# *.sink.console.period. # 4. A metrics specific configuration # "spark.metrics.conf=${SPARK_HOME}/conf/metrics.properties" should be -# added to Java property using -Dspark.metrics.conf=xxx if you want to -# customize metrics system, or you can put it in ${SPARK_HOME}/conf, -# metrics system will search and load it automatically. +# added to Java properties using -Dspark.metrics.conf=xxx if you want to +# customize metrics system. You can also put the file in ${SPARK_HOME}/conf +# and it will be loaded automatically. # Enable JmxSink for all instances by class name #*.sink.jmx.class=spark.metrics.sink.JmxSink From e466a55a6b803a5295e6dcc106a4abef917e7058 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 1 Aug 2013 15:45:21 -0700 Subject: [PATCH 400/419] Revert Mesos version to 0.9 since the 0.12 artifact has target Java 7 --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index bb96ad4ae3..c822f49e78 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -178,7 +178,7 @@ object SparkBuild extends Build { "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", - "org.apache.mesos" % "mesos" % "0.12.0-incubating", + "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", From b3ae5b25d5e20c422049dc0cada397e51f52cb32 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 2 Aug 2013 13:25:14 -0700 Subject: [PATCH 401/419] Shows time the app has been running --- core/src/main/scala/spark/SparkContext.scala | 2 ++ core/src/main/scala/spark/ui/jobs/IndexPage.scala | 6 +++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 0d1f9fa8d4..ca5ea30d22 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -128,6 +128,8 @@ class SparkContext( private[spark] val ui = new SparkUI(this) ui.bind() + val startTime = System.currentTimeMillis() + // Add each JAR given through the constructor if (jars != null) { jars.foreach { addJar(_) } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 4ad787565d..12166f10c2 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -54,7 +54,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { val summary: NodeSeq =
      -
    • +
    • + Duration: + {parent.formatDuration(now - listener.sc.startTime)} +
    • +
    • CPU time: {parent.formatDuration(listener.totalTime + activeTime)}
    • From 5b3784a79c4e6069ace17c5dddc0ad0046909c8b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 2 Aug 2013 15:46:09 -0700 Subject: [PATCH 402/419] Show user-defined job name in UI --- .../main/resources/spark/ui/static/webui.css | 4 ++++ core/src/main/scala/spark/SparkContext.scala | 11 +++++++++-- .../main/scala/spark/scheduler/JobLogger.scala | 5 +++-- .../scala/spark/ui/UIWorkloadGenerator.scala | 6 ++---- .../spark/ui/jobs/JobProgressListener.scala | 18 +++++++++++++----- .../main/scala/spark/ui/jobs/StageTable.scala | 10 +++++++--- .../spark/scheduler/LocalSchedulerSuite.scala | 2 +- 7 files changed, 39 insertions(+), 17 deletions(-) diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css index f7537bb766..8b9f4ee938 100644 --- a/core/src/main/resources/spark/ui/static/webui.css +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -47,3 +47,7 @@ padding-top: 7px; padding-left: 4px; } + +.table td { + vertical-align: middle !important; +} diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 97e1aaf49e..039f5522b7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -265,12 +265,18 @@ class SparkContext( localProperties.value = new Properties() } - def addLocalProperties(key: String, value: String) { + def addLocalProperty(key: String, value: String) { if(localProperties.value == null) { localProperties.value = new Properties() } localProperties.value.setProperty(key,value) } + + /** Set a human readable description of the current job. */ + def setDescription(value: String) { + addLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value) + } + // Post init taskScheduler.postStartHook() @@ -841,6 +847,7 @@ class SparkContext( * various Spark features. */ object SparkContext { + val SPARK_JOB_DESCRIPTION = "spark.job.description" implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 @@ -958,7 +965,6 @@ object SparkContext { } } - /** * A class encapsulating how to convert some type T to Writable. It stores both the Writable class * corresponding to T (e.g. IntWritable for Int) and a function for doing the conversion. @@ -970,3 +976,4 @@ private[spark] class WritableConverter[T]( val writableClass: ClassManifest[T] => Class[_ <: Writable], val convert: Writable => T) extends Serializable + diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index c7e8f8a9a1..ad2efcec63 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -26,6 +26,7 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{Map, HashMap, ListBuffer} import scala.io.Source import spark._ +import spark.SparkContext import spark.executor.TaskMetrics import spark.scheduler.cluster.TaskInfo @@ -317,8 +318,8 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { protected def recordJobProperties(jobID: Int, properties: Properties) { if(properties != null) { - val annotation = properties.getProperty("spark.job.annotation", "") - jobLogInfo(jobID, annotation, false) + val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") + jobLogInfo(jobID, description, false) } } diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 3ac35085eb..97ea644021 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -46,13 +46,11 @@ private[spark] object UIWorkloadGenerator { } val sc = new SparkContext(master, appName) - // NOTE: Right now there is no easy way for us to show spark.job.annotation for a given phase, - // but we pass it here anyways since it will be useful once we do. def setProperties(s: String) = { if(schedulingMode == SchedulingMode.FAIR) { - sc.addLocalProperties("spark.scheduler.cluster.fair.pool", s) + sc.addLocalProperty("spark.scheduler.cluster.fair.pool", s) } - sc.addLocalProperties("spark.job.annotation", s) + sc.addLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, s) } val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala index 200e13cf99..f22c4e39e3 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala @@ -15,6 +15,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList val DEFAULT_POOL_NAME = "default" val stageToPool = new HashMap[Stage, String]() + val stageToDescription = new HashMap[Stage, String]() val poolToActiveStages = new HashMap[String, HashSet[Stage]]() val activeStages = HashSet[Stage]() @@ -57,6 +58,8 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList stageToTasksActive.remove(s.id) stageToTasksComplete.remove(s.id) stageToTasksFailed.remove(s.id) + stageToPool.remove(s) + if (stageToDescription.contains(s)) {stageToDescription.remove(s)} }) stages.trimEnd(toRemove) } @@ -66,12 +69,17 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { val stage = stageSubmitted.stage activeStages += stage - var poolName = DEFAULT_POOL_NAME - if (stageSubmitted.properties != null) { - poolName = stageSubmitted.properties.getProperty("spark.scheduler.cluster.fair.pool", - DEFAULT_POOL_NAME) - } + + val poolName = Option(stageSubmitted.properties).map { + p => p.getProperty("spark.scheduler.cluster.fair.pool", DEFAULT_POOL_NAME) + }.getOrElse(DEFAULT_POOL_NAME) stageToPool(stage) = poolName + + val description = Option(stageSubmitted.properties).flatMap { + p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) + } + description.map(d => stageToDescription(stage) = d) + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) stages += stage } diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 3257f4e360..38fa3bcbcd 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -34,7 +34,7 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU
    Stage IdPool NameOriginDescription Submitted Duration Tasks: Complete/Total
    {s.id}{poolName.get}{s.name}{submissionTime}{description}{submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} {makeProgressBar(startedTasks, completedTasks, totalTasks)}

    Back to Master