From e6d4a74d2d92345985c1603f9b526a6347adb7cf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 10 Apr 2014 02:10:40 -0700 Subject: [PATCH 01/61] Revert "SPARK-729: Closures not always serialized at capture time" This reverts commit 8ca3b2bc90a63b23a03f339e390174cd7a672b40. --- .../scala/org/apache/spark/SparkContext.scala | 16 +--- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../apache/spark/util/ClosureCleaner.scala | 21 +---- .../scala/org/apache/spark/FailureSuite.scala | 17 +--- .../ProactiveClosureSerializationSuite.scala | 94 ------------------- .../spark/util/ClosureCleanerSuite.scala | 68 -------------- .../org/apache/spark/graphx/GraphSuite.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 8 +- 8 files changed, 14 insertions(+), 218 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 545807ffbce55..76305237b03d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1002,9 +1002,7 @@ class SparkContext(config: SparkConf) extends Logging { require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") } val callSite = getCallSite - // There's no need to check this function for serializability, - // since it will be run right away. - val cleanedFunc = clean(func, false) + val cleanedFunc = clean(func) logInfo("Starting job: " + callSite) val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, @@ -1137,18 +1135,14 @@ class SparkContext(config: SparkConf) extends Logging { def cancelAllJobs() { dagScheduler.cancelAllJobs() } - + /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) - * - * @param f closure to be cleaned and optionally serialized - * @param captureNow whether or not to serialize this closure and capture any free - * variables immediately; defaults to true. If this is set and f is not serializable, - * it will raise an exception. */ - private[spark] def clean[F <: AnyRef : ClassTag](f: F, captureNow: Boolean = true): F = { - ClosureCleaner.clean(f, captureNow) + private[spark] def clean[F <: AnyRef](f: F): F = { + ClosureCleaner.clean(f) + f } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e363ea777d8eb..3437b2cac19c2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -660,16 +660,14 @@ abstract class RDD[T: ClassTag]( * Applies a function f to all elements of this RDD. */ def foreach(f: T => Unit) { - val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) + sc.runJob(this, (iter: Iterator[T]) => iter.foreach(f)) } /** * Applies a function f to each partition of this RDD. */ def foreachPartition(f: Iterator[T] => Unit) { - val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) + sc.runJob(this, (iter: Iterator[T]) => f(iter)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index e474b1a850d65..cdbbc65292188 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -22,14 +22,10 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.Map import scala.collection.mutable.Set -import scala.reflect.ClassTag - import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ import org.apache.spark.Logging -import org.apache.spark.SparkEnv -import org.apache.spark.SparkException private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -105,7 +101,7 @@ private[spark] object ClosureCleaner extends Logging { } } - def clean[F <: AnyRef : ClassTag](func: F, captureNow: Boolean = true): F = { + def clean(func: AnyRef) { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) @@ -154,21 +150,6 @@ private[spark] object ClosureCleaner extends Logging { field.setAccessible(true) field.set(func, outer) } - - if (captureNow) { - cloneViaSerializing(func) - } else { - func - } - } - - private def cloneViaSerializing[T: ClassTag](func: T): T = { - try { - val serializer = SparkEnv.get.closureSerializer.newInstance() - serializer.deserialize[T](serializer.serialize[T](func)) - } catch { - case ex: Exception => throw new SparkException("Task not serializable: " + ex.toString) - } } private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 4f9300419e6f8..12dbebcb28644 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -107,7 +107,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } - test("failure because closure in final-stage task is not serializable") { + test("failure because task closure is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable @@ -118,13 +118,6 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) - FailureSuiteState.clear() - } - - test("failure because closure in early-stage task is not serializable") { - sc = new SparkContext("local[1,1]", "test") - val a = new NonSerializable - // 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() @@ -132,13 +125,6 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) - FailureSuiteState.clear() - } - - test("failure because closure in foreach task is not serializable") { - sc = new SparkContext("local[1,1]", "test") - val a = new NonSerializable - // Non-serializable closure in foreach function val thrown2 = intercept[SparkException] { sc.parallelize(1 to 10, 2).foreach(x => println(a)) @@ -149,6 +135,5 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } - // TODO: Need to add tests with shuffle fetch failures. } diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala deleted file mode 100644 index 76662264e7e94..0000000000000 --- a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.serializer; - -import java.io.NotSerializableException - -import org.scalatest.FunSuite - -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkException -import org.apache.spark.SharedSparkContext - -/* A trivial (but unserializable) container for trivial functions */ -class UnserializableClass { - def op[T](x: T) = x.toString - - def pred[T](x: T) = x.toString.length % 2 == 0 -} - -class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { - - def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) - - test("throws expected serialization exceptions on actions") { - val (data, uc) = fixture - - val ex = intercept[SparkException] { - data.map(uc.op(_)).count - } - - assert(ex.getMessage.matches(".*Task not serializable.*")) - } - - // There is probably a cleaner way to eliminate boilerplate here, but we're - // iterating over a map from transformation names to functions that perform that - // transformation on a given RDD, creating one test case for each - - for (transformation <- - Map("map" -> map _, "flatMap" -> flatMap _, "filter" -> filter _, "mapWith" -> mapWith _, - "mapPartitions" -> mapPartitions _, "mapPartitionsWithIndex" -> mapPartitionsWithIndex _, - "mapPartitionsWithContext" -> mapPartitionsWithContext _, "filterWith" -> filterWith _)) { - val (name, xf) = transformation - - test(s"$name transformations throw proactive serialization exceptions") { - val (data, uc) = fixture - - val ex = intercept[SparkException] { - xf(data, uc) - } - - assert(ex.getMessage.matches(".*Task not serializable.*"), s"RDD.$name doesn't proactively throw NotSerializableException") - } - } - - def map(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.map(y => uc.op(y)) - - def mapWith(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapWith(x => x.toString)((x,y) => x + uc.op(y)) - - def flatMap(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.flatMap(y=>Seq(uc.op(y))) - - def filter(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.filter(y=>uc.pred(y)) - - def filterWith(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.filterWith(x => x.toString)((x,y) => uc.pred(y)) - - def mapPartitions(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitions(_.map(y => uc.op(y))) - - def mapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitionsWithIndex((_, it) => it.map(y => uc.op(y))) - - def mapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitionsWithContext((_, it) => it.map(y => uc.op(y))) - -} diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index c635da6cacd70..439e5644e20a3 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -50,27 +50,6 @@ class ClosureCleanerSuite extends FunSuite { val obj = new TestClassWithNesting(1) assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 } - - test("capturing free variables in closures at RDD definition") { - val obj = new TestCaptureVarClass() - val (ones, onesPlusZeroes) = obj.run() - - assert(ones === onesPlusZeroes) - } - - test("capturing free variable fields in closures at RDD definition") { - val obj = new TestCaptureFieldClass() - val (ones, onesPlusZeroes) = obj.run() - - assert(ones === onesPlusZeroes) - } - - test("capturing arrays in closures at RDD definition") { - val obj = new TestCaptureArrayEltClass() - val (observed, expected) = obj.run() - - assert(observed === expected) - } } // A non-serializable class we create in closures to make sure that we aren't @@ -164,50 +143,3 @@ class TestClassWithNesting(val y: Int) extends Serializable { } } } - -class TestCaptureFieldClass extends Serializable { - class ZeroBox extends Serializable { - var zero = 0 - } - - def run(): (Int, Int) = { - val zb = new ZeroBox - - withSpark(new SparkContext("local", "test")) {sc => - val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) - val onesPlusZeroes = ones.map(_ + zb.zero) - - zb.zero = 5 - - (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) - } - } -} - -class TestCaptureArrayEltClass extends Serializable { - def run(): (Int, Int) = { - withSpark(new SparkContext("local", "test")) {sc => - val rdd = sc.parallelize(1 to 10) - val data = Array(1, 2, 3) - val expected = data(0) - val mapped = rdd.map(x => data(0)) - data(0) = 4 - (mapped.first, expected) - } - } -} - -class TestCaptureVarClass extends Serializable { - def run(): (Int, Int) = { - var zero = 0 - - withSpark(new SparkContext("local", "test")) {sc => - val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) - val onesPlusZeroes = ones.map(_ + zero) - - zero = 5 - - (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) - } - } -} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index c65e36636fe10..28d34dd9a1a41 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -62,7 +62,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) - graph.triplets.collect.map { et => + graph.triplets.map { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 4759b629a9931..d043200f71a0b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -539,7 +539,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = { - transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r), false)) + transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r))) } /** @@ -547,7 +547,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { - val cleanedF = context.sparkContext.clean(transformFunc, false) + val cleanedF = context.sparkContext.clean(transformFunc) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 1) cleanedF(rdds.head.asInstanceOf[RDD[T]], time) @@ -562,7 +562,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc, false) + val cleanedF = ssc.sparkContext.clean(transformFunc) transformWith(other, (rdd1: RDD[T], rdd2: RDD[U], time: Time) => cleanedF(rdd1, rdd2)) } @@ -573,7 +573,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc, false) + val cleanedF = ssc.sparkContext.clean(transformFunc) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 2) val rdd1 = rdds(0).asInstanceOf[RDD[T]] From a74fbbbca8f0d89b2e0e4e8751a93d33efc4fa9e Mon Sep 17 00:00:00 2001 From: witgo Date: Thu, 10 Apr 2014 10:35:24 -0700 Subject: [PATCH 02/61] Fix SPARK-1413: Parquet messes up stdout and stdin when used in Spark REPL Author: witgo Closes #325 from witgo/SPARK-1413 and squashes the following commits: e57cd8e [witgo] use scala reflection to access and call the SLF4JBridgeHandler methods 45c8f40 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 5e35d87 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 0d5f819 [witgo] review commit 45e5b70 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 fa69dcf [witgo] Merge branch 'master' into SPARK-1413 3c98dc4 [witgo] Merge branch 'master' into SPARK-1413 38160cb [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 ba09bcd [witgo] remove set the parquet log level a63d574 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 5231ecd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 3feb635 [witgo] parquet logger use parent handler fa00d5d [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 8bb6ffd [witgo] enableLogForwarding note fix edd9630 [witgo] move to f447f50 [witgo] merging master 5ad52bd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 76670c1 [witgo] review commit 70f3c64 [witgo] Fix SPARK-1413 --- .../main/scala/org/apache/spark/Logging.scala | 20 +++++++++--- .../spark/sql/parquet/ParquetRelation.scala | 31 +++++-------------- 2 files changed, 23 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index e5e15617acb10..9d429dceeb858 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. - * + * * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. * This will likely be changed or removed in future releases. */ @@ -60,7 +60,7 @@ trait Logging { protected def logDebug(msg: => String) { if (log.isDebugEnabled) log.debug(msg) } - + protected def logTrace(msg: => String) { if (log.isTraceEnabled) log.trace(msg) } @@ -117,10 +117,10 @@ trait Logging { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" val classLoader = this.getClass.getClassLoader Option(classLoader.getResource(defaultLogProps)) match { - case Some(url) => + case Some(url) => PropertyConfigurator.configure(url) log.info(s"Using Spark's default log4j profile: $defaultLogProps") - case None => + case None => System.err.println(s"Spark was unable to load $defaultLogProps") } } @@ -135,4 +135,16 @@ trait Logging { private object Logging { @volatile private var initialized = false val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 505ad0a2c77c1..4d7c86a3a4fc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -82,30 +82,13 @@ private[sql] case class ParquetRelation(val path: String) private[sql] object ParquetRelation { def enableLogForwarding() { - // Note: Parquet does not use forwarding to parent loggers which - // is required for the JUL-SLF4J bridge to work. Also there is - // a default logger that appends to Console which needs to be - // reset. - import org.slf4j.bridge.SLF4JBridgeHandler - import java.util.logging.Logger - import java.util.logging.LogManager - - val loggerNames = Seq( - "parquet.hadoop.ColumnChunkPageWriteStore", - "parquet.hadoop.InternalParquetRecordWriter", - "parquet.hadoop.ParquetRecordReader", - "parquet.hadoop.ParquetInputFormat", - "parquet.hadoop.ParquetOutputFormat", - "parquet.hadoop.ParquetFileReader", - "parquet.hadoop.InternalParquetRecordReader", - "parquet.hadoop.codec.CodecConfig") - LogManager.getLogManager.reset() - SLF4JBridgeHandler.install() - for(name <- loggerNames) { - val logger = Logger.getLogger(name) - logger.setParent(Logger.getLogger(Logger.GLOBAL_LOGGER_NAME)) - logger.setUseParentHandlers(true) - } + // Note: Logger.getLogger("parquet") has a default logger + // that appends to Console which needs to be cleared. + val parquetLogger = java.util.logging.Logger.getLogger("parquet") + parquetLogger.getHandlers.foreach(parquetLogger.removeHandler) + // TODO(witgo): Need to set the log level ? + // if(parquetLogger.getLevel != null) parquetLogger.setLevel(null) + if (!parquetLogger.getUseParentHandlers) parquetLogger.setUseParentHandlers(true) } // The element type for the RDDs that this relation maps to. From 79820fe825ed7c09d55f50503b7ab53d4585e5f7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 10 Apr 2014 10:39:34 -0700 Subject: [PATCH 03/61] [SPARK-1276] Add a HistoryServer to render persisted UI The new feature of event logging, introduced in #42, allows the user to persist the details of his/her Spark application to storage, and later replay these events to reconstruct an after-the-fact SparkUI. Currently, however, a persisted UI can only be rendered through the standalone Master. This greatly limits the use case of this new feature as many people also run Spark on Yarn / Mesos. This PR introduces a new entity called the HistoryServer, which, given a log directory, keeps track of all completed applications independently of a Spark Master. Unlike Master, the HistoryServer needs not be running while the application is still running. It is relatively light-weight in that it only maintains static information of applications and performs no scheduling. To quickly test it out, generate event logs with ```spark.eventLog.enabled=true``` and run ```sbin/start-history-server.sh ```. Your HistoryServer awaits on port 18080. Comments and feedback are most welcome. --- A few other changes introduced in this PR include refactoring the WebUI interface, which is beginning to have a lot of duplicate code now that we have added more functionality to it. Two new SparkListenerEvents have been introduced (SparkListenerApplicationStart/End) to keep track of application name and start/finish times. This PR also clarifies the semantics of the ReplayListenerBus introduced in #42. A potential TODO in the future (not part of this PR) is to render live applications in addition to just completed applications. This is useful when applications fail, a condition that our current HistoryServer does not handle unless the user manually signals application completion (by creating the APPLICATION_COMPLETION file). Handling live applications becomes significantly more challenging, however, because it is now necessary to render the same SparkUI multiple times. To avoid reading the entire log every time, which is inefficient, we must handle reading the log from where we previously left off, but this becomes fairly complicated because we must deal with the arbitrary behavior of each input stream. Author: Andrew Or Closes #204 from andrewor14/master and squashes the following commits: 7b7234c [Andrew Or] Finished -> Completed b158d98 [Andrew Or] Address Patrick's comments 69d1b41 [Andrew Or] Do not block on posting SparkListenerApplicationEnd 19d5dd0 [Andrew Or] Merge github.com:apache/spark f7f5bf0 [Andrew Or] Make history server's web UI port a Spark configuration 2dfb494 [Andrew Or] Decouple checking for application completion from replaying d02dbaa [Andrew Or] Expose Spark version and include it in event logs 2282300 [Andrew Or] Add documentation for the HistoryServer 567474a [Andrew Or] Merge github.com:apache/spark 6edf052 [Andrew Or] Merge github.com:apache/spark 19e1fb4 [Andrew Or] Address Thomas' comments 248cb3d [Andrew Or] Limit number of live applications + add configurability a3598de [Andrew Or] Do not close file system with ReplayBus + fix bind address bc46fc8 [Andrew Or] Merge github.com:apache/spark e2f4ff9 [Andrew Or] Merge github.com:apache/spark 050419e [Andrew Or] Merge github.com:apache/spark 81b568b [Andrew Or] Fix strange error messages... 0670743 [Andrew Or] Decouple page rendering from loading files from disk 1b2f391 [Andrew Or] Minor changes a9eae7e [Andrew Or] Merge branch 'master' of github.com:apache/spark d5154da [Andrew Or] Styling and comments 5dbfbb4 [Andrew Or] Merge branch 'master' of github.com:apache/spark 60bc6d5 [Andrew Or] First complete implementation of HistoryServer (only for finished apps) 7584418 [Andrew Or] Report application start/end times to HistoryServer 8aac163 [Andrew Or] Add basic application table c086bd5 [Andrew Or] Add HistoryServer and scripts ++ Refactor WebUI interface --- bin/spark-class | 8 +- bin/spark-class2.cmd | 7 +- .../scala/org/apache/spark/SparkContext.scala | 26 +- .../spark/deploy/ApplicationDescription.scala | 4 +- .../spark/deploy/SparkUIContainer.scala | 50 +++ .../spark/deploy/history/HistoryServer.scala | 287 ++++++++++++++++++ .../history/HistoryServerArguments.scala | 76 +++++ .../spark/deploy/history/IndexPage.scala | 82 +++++ .../apache/spark/deploy/master/Master.scala | 62 ++-- .../spark/deploy/master/ui/MasterWebUI.scala | 43 +-- .../spark/deploy/worker/ui/WorkerWebUI.scala | 22 +- .../scheduler/ApplicationEventListener.scala | 50 +++ .../scheduler/EventLoggingListener.scala | 146 ++++++++- .../spark/scheduler/ReplayListenerBus.scala | 65 ++-- .../spark/scheduler/SparkListener.scala | 15 + .../spark/scheduler/SparkListenerBus.scala | 4 + .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../apache/spark/storage/FileSegment.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 49 +-- .../scala/org/apache/spark/ui/WebUI.scala | 21 +- .../apache/spark/ui/env/EnvironmentUI.scala | 3 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 3 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 3 +- .../apache/spark/ui/jobs/JobProgressUI.scala | 3 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 3 +- .../spark/ui/storage/BlockManagerUI.scala | 3 +- .../apache/spark/ui/storage/IndexPage.scala | 3 +- .../org/apache/spark/ui/storage/RDDPage.scala | 3 +- .../org/apache/spark/util/FileLogger.scala | 27 +- .../org/apache/spark/util/JsonProtocol.scala | 31 ++ .../scala/org/apache/spark/util/Utils.scala | 8 +- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 24 +- docs/monitoring.md | 70 ++++- .../apache/spark/repl/SparkILoopInit.scala | 4 +- sbin/start-history-server.sh | 37 +++ sbin/stop-history-server.sh | 25 ++ 38 files changed, 1075 insertions(+), 201 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala create mode 100755 sbin/start-history-server.sh create mode 100755 sbin/stop-history-server.sh diff --git a/bin/spark-class b/bin/spark-class index 76fde3e448891..1b0d309cc5b1c 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -47,9 +47,9 @@ DEFAULT_MEM=${SPARK_MEM:-512m} SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" -# Add java opts and memory settings for master, worker, executors, and repl. +# Add java opts and memory settings for master, worker, history server, executors, and repl. case "$1" in - # Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. + # Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. 'org.apache.spark.deploy.master.Master') OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} @@ -58,6 +58,10 @@ case "$1" in OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} ;; + 'org.apache.spark.deploy.history.HistoryServer') + OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_HISTORY_OPTS" + OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} + ;; # Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. 'org.apache.spark.executor.CoarseGrainedExecutorBackend') diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index f488cfdbeceb6..4302c1b6b7ff4 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -45,14 +45,17 @@ if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true -rem Add java opts and memory settings for master, worker, executors, and repl. -rem Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. +rem Add java opts and memory settings for master, worker, history server, executors, and repl. +rem Master, Worker and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. if "%1"=="org.apache.spark.deploy.master.Master" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% ) else if "%1"=="org.apache.spark.deploy.worker.Worker" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% +) else if "%1"=="org.apache.spark.deploy.history.HistoryServer" ( + set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_HISTORY_OPTS% + if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. ) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" ( diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 76305237b03d5..e6c9b7000d819 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -219,15 +219,12 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val eventLogger: Option[EventLoggingListener] = { if (conf.getBoolean("spark.eventLog.enabled", false)) { val logger = new EventLoggingListener(appName, conf) + logger.start() listenerBus.addListener(logger) Some(logger) } else None } - // Information needed to replay logged events, if any - private[spark] val eventLoggingInfo: Option[EventLoggingInfo] = - eventLogger.map { logger => Some(logger.info) }.getOrElse(None) - // At this point, all relevant SparkListeners have been registered, so begin releasing events listenerBus.start() @@ -292,6 +289,7 @@ class SparkContext(config: SparkConf) extends Logging { cleaner.foreach(_.start()) postEnvironmentUpdate() + postApplicationStart() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration: Configuration = { @@ -777,6 +775,9 @@ class SparkContext(config: SparkConf) extends Logging { listenerBus.addListener(listener) } + /** The version of Spark on which this application is running. */ + def version = SparkContext.SPARK_VERSION + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. @@ -930,6 +931,7 @@ class SparkContext(config: SparkConf) extends Logging { /** Shut down the SparkContext. */ def stop() { + postApplicationEnd() ui.stop() // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. @@ -1175,6 +1177,20 @@ class SparkContext(config: SparkConf) extends Logging { /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Post the application start event */ + private def postApplicationStart() { + listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) + } + + /** + * Post the application end event to all listeners immediately, rather than adding it + * to the event queue for it to be asynchronously processed eventually. Otherwise, a race + * condition exists in which the listeners may stop before this event has been propagated. + */ + private def postApplicationEnd() { + listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis)) + } + /** Post the environment update event once the task scheduler is ready */ private def postEnvironmentUpdate() { if (taskScheduler != null) { @@ -1200,6 +1216,8 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { + private[spark] val SPARK_VERSION = "1.0.0" + private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 15fa8a7679874..86305d2ea8a09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import org.apache.spark.scheduler.EventLoggingInfo - private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], @@ -26,7 +24,7 @@ private[spark] class ApplicationDescription( val command: Command, val sparkHome: Option[String], var appUiUrl: String, - val eventLogInfo: Option[EventLoggingInfo] = None) + val eventLogDir: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala new file mode 100644 index 0000000000000..33fceae4ff489 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import org.apache.spark.ui.{SparkUI, WebUI} + +private[spark] abstract class SparkUIContainer(name: String) extends WebUI(name) { + + /** Attach a SparkUI to this container. Only valid after bind(). */ + def attachUI(ui: SparkUI) { + assert(serverInfo.isDefined, + "%s must be bound to a server before attaching SparkUIs".format(name)) + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } + + /** Detach a SparkUI from this container. Only valid after bind(). */ + def detachUI(ui: SparkUI) { + assert(serverInfo.isDefined, + "%s must be bound to a server before detaching SparkUIs".format(name)) + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + if (handler.isStarted) { + handler.stop() + } + rootHandler.removeHandler(handler) + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala new file mode 100644 index 0000000000000..97d2ba9deed33 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.eclipse.jetty.servlet.ServletContextHandler + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.SparkUIContainer +import org.apache.spark.scheduler._ +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils + +/** + * A web server that renders SparkUIs of completed applications. + * + * For the standalone mode, MasterWebUI already achieves this functionality. Thus, the + * main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos). + * + * The logging directory structure is as follows: Within the given base directory, each + * application's event logs are maintained in the application's own sub-directory. This + * is the same structure as maintained in the event log write code path in + * EventLoggingListener. + * + * @param baseLogDir The base directory in which event logs are found + */ +class HistoryServer( + val baseLogDir: String, + conf: SparkConf) + extends SparkUIContainer("History Server") with Logging { + + import HistoryServer._ + + private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) + private val localHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) + private val port = WEB_UI_PORT + private val securityManager = new SecurityManager(conf) + private val indexPage = new IndexPage(this) + + // A timestamp of when the disk was last accessed to check for log updates + private var lastLogCheckTime = -1L + + // Number of completed applications found in this directory + private var numCompletedApplications = 0 + + @volatile private var stopped = false + + /** + * A background thread that periodically checks for event log updates on disk. + * + * If a log check is invoked manually in the middle of a period, this thread re-adjusts the + * time at which it performs the next log check to maintain the same period as before. + * + * TODO: Add a mechanism to update manually. + */ + private val logCheckingThread = new Thread { + override def run() { + while (!stopped) { + val now = System.currentTimeMillis + if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { + checkForLogs() + Thread.sleep(UPDATE_INTERVAL_MS) + } else { + // If the user has manually checked for logs recently, wait until + // UPDATE_INTERVAL_MS after the last check time + Thread.sleep(lastLogCheckTime + UPDATE_INTERVAL_MS - now) + } + } + } + } + + private val handlers = Seq[ServletContextHandler]( + createStaticHandler(STATIC_RESOURCE_DIR, "/static"), + createServletHandler("/", + (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager) + ) + + // A mapping of application ID to its history information, which includes the rendered UI + val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + + /** + * Start the history server. + * + * This starts a background thread that periodically synchronizes information displayed on + * this UI with the event logs in the provided base directory. + */ + def start() { + logCheckingThread.start() + } + + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) + } catch { + case e: Exception => + logError("Failed to bind HistoryServer", e) + System.exit(1) + } + } + + /** + * Check for any updates to event logs in the base directory. This is only effective once + * the server has been bound. + * + * If a new completed application is found, the server renders the associated SparkUI + * from the application's event logs, attaches this UI to itself, and stores metadata + * information for this application. + * + * If the logs for an existing completed application are no longer found, the server + * removes all associated information and detaches the SparkUI. + */ + def checkForLogs() = synchronized { + if (serverInfo.isDefined) { + lastLogCheckTime = System.currentTimeMillis + logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTime)) + try { + val logStatus = fileSystem.listStatus(new Path(baseLogDir)) + val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + val logInfos = logDirs + .sortBy { dir => getModificationTime(dir) } + .map { dir => (dir, EventLoggingListener.parseLoggingInfo(dir.getPath, fileSystem)) } + .filter { case (dir, info) => info.applicationComplete } + + // Logging information for applications that should be retained + val retainedLogInfos = logInfos.takeRight(RETAINED_APPLICATIONS) + val retainedAppIds = retainedLogInfos.map { case (dir, _) => dir.getPath.getName } + + // Remove any applications that should no longer be retained + appIdToInfo.foreach { case (appId, info) => + if (!retainedAppIds.contains(appId)) { + detachUI(info.ui) + appIdToInfo.remove(appId) + } + } + + // Render the application's UI if it is not already there + retainedLogInfos.foreach { case (dir, info) => + val appId = dir.getPath.getName + if (!appIdToInfo.contains(appId)) { + renderSparkUI(dir, info) + } + } + + // Track the total number of completed applications observed this round + numCompletedApplications = logInfos.size + + } catch { + case t: Throwable => logError("Exception in checking for event log updates", t) + } + } else { + logWarning("Attempted to check for event log updates before binding the server.") + } + } + + /** + * Render a new SparkUI from the event logs if the associated application is completed. + * + * HistoryServer looks for a special file that indicates application completion in the given + * directory. If this file exists, the associated application is regarded to be completed, in + * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. + */ + private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) { + val path = logDir.getPath + val appId = path.getName + val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) + val ui = new SparkUI(replayBus, appId, "/history/" + appId) + val appListener = new ApplicationEventListener + replayBus.addListener(appListener) + + // Do not call ui.bind() to avoid creating a new server for each application + ui.start() + replayBus.replay() + if (appListener.applicationStarted) { + attachUI(ui) + val appName = appListener.appName + val sparkUser = appListener.sparkUser + val startTime = appListener.startTime + val endTime = appListener.endTime + val lastUpdated = getModificationTime(logDir) + ui.setAppName(appName + " (completed)") + appIdToInfo(appId) = ApplicationHistoryInfo(appId, appName, startTime, endTime, + lastUpdated, sparkUser, path, ui) + } + } + + /** Stop the server and close the file system. */ + override def stop() { + super.stop() + stopped = true + fileSystem.close() + } + + /** Return the address of this server. */ + def getAddress: String = "http://" + publicHost + ":" + boundPort + + /** Return the number of completed applications found, whether or not the UI is rendered. */ + def getNumApplications: Int = numCompletedApplications + + /** Return when this directory was last modified. */ + private def getModificationTime(dir: FileStatus): Long = { + try { + val logFiles = fileSystem.listStatus(dir.getPath) + if (logFiles != null && !logFiles.isEmpty) { + logFiles.map(_.getModificationTime).max + } else { + dir.getModificationTime + } + } catch { + case t: Throwable => + logError("Exception in accessing modification time of %s".format(dir.getPath), t) + -1L + } + } +} + +/** + * The recommended way of starting and stopping a HistoryServer is through the scripts + * start-history-server.sh and stop-history-server.sh. The path to a base log directory + * is must be specified, while the requested UI port is optional. For example: + * + * ./sbin/spark-history-server.sh /tmp/spark-events + * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events + * + * This launches the HistoryServer as a Spark daemon. + */ +object HistoryServer { + private val conf = new SparkConf + + // Interval between each check for event log updates + val UPDATE_INTERVAL_MS = conf.getInt("spark.history.updateInterval", 10) * 1000 + + // How many applications to retain + val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250) + + // The port to which the web UI is bound + val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) + + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR + + def main(argStrings: Array[String]) { + val args = new HistoryServerArguments(argStrings) + val server = new HistoryServer(args.logDir, conf) + server.bind() + server.start() + + // Wait until the end of the world... or if the HistoryServer process is manually stopped + while(true) { Thread.sleep(Int.MaxValue) } + server.stop() + } +} + + +private[spark] case class ApplicationHistoryInfo( + id: String, + name: String, + startTime: Long, + endTime: Long, + lastUpdated: Long, + sparkUser: String, + logDirPath: Path, + ui: SparkUI) { + def started = startTime != -1 + def completed = endTime != -1 +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala new file mode 100644 index 0000000000000..943c061743dbd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import java.net.URI + +import org.apache.hadoop.fs.Path + +import org.apache.spark.util.Utils + +/** + * Command-line parser for the master. + */ +private[spark] class HistoryServerArguments(args: Array[String]) { + var logDir = "" + + parse(args.toList) + + private def parse(args: List[String]): Unit = { + args match { + case ("--dir" | "-d") :: value :: tail => + logDir = value + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case Nil => + + case _ => + printUsageAndExit(1) + } + validateLogDir() + } + + private def validateLogDir() { + if (logDir == "") { + System.err.println("Logging directory must be specified.") + printUsageAndExit(1) + } + val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + val path = new Path(logDir) + if (!fileSystem.exists(path)) { + System.err.println("Logging directory specified does not exist: %s".format(logDir)) + printUsageAndExit(1) + } + if (!fileSystem.getFileStatus(path).isDir) { + System.err.println("Logging directory specified is not a directory: %s".format(logDir)) + printUsageAndExit(1) + } + } + + private def printUsageAndExit(exitCode: Int) { + System.err.println( + "Usage: HistoryServer [options]\n" + + "\n" + + "Options:\n" + + " -d DIR, --dir DIR Location of event log files") + System.exit(exitCode) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala new file mode 100644 index 0000000000000..54dffffec71c5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.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 org.apache.spark.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{UIUtils, WebUI} + +private[spark] class IndexPage(parent: HistoryServer) { + + def render(request: HttpServletRequest): Seq[Node] = { + val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } + val appTable = UIUtils.listingTable(appHeader, appRow, appRows) + val content = +
+
+
    +
  • Event Log Location: {parent.baseLogDir}
  • +
+ { + if (parent.appIdToInfo.size > 0) { +

+ Showing {parent.appIdToInfo.size}/{parent.getNumApplications} + Completed Application{if (parent.getNumApplications > 1) "s" else ""} +

++ + appTable + } else { +

No Completed Applications Found

+ } + } +
+
+ UIUtils.basicSparkPage(content, "History Server") + } + + private val appHeader = Seq( + "App Name", + "Started", + "Completed", + "Duration", + "Spark User", + "Log Directory", + "Last Updated") + + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { + val appName = if (info.started) info.name else info.logDirPath.getName + val uiAddress = parent.getAddress + info.ui.basePath + val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" + val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not completed" + val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L + val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" + val sparkUser = if (info.started) info.sparkUser else "Unknown user" + val logDirectory = info.logDirPath.getName + val lastUpdated = WebUI.formatDate(info.lastUpdated) + + {appName} + {startTime} + {endTime} + {duration} + {sparkUser} + {logDirectory} + {lastUpdated} + + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 95bd62e88db2b..2446e86cb6672 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -29,6 +29,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension +import org.apache.hadoop.fs.FileSystem import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} @@ -37,7 +38,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} @@ -45,7 +46,8 @@ private[spark] class Master( host: String, port: Int, webUiPort: Int, - val securityMgr: SecurityManager) extends Actor with Logging { + val securityMgr: SecurityManager) + extends Actor with Logging { import context.dispatcher // to use Akka's scheduler.schedule() @@ -71,6 +73,7 @@ private[spark] class Master( var nextAppNumber = 0 val appIdToUI = new HashMap[String, SparkUI] + val fileSystemsUsed = new HashSet[FileSystem] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -149,6 +152,7 @@ private[spark] class Master( override def postStop() { webUi.stop() + fileSystemsUsed.foreach(_.close()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -630,11 +634,7 @@ private[spark] class Master( waitingApps -= app // If application events are logged, use them to rebuild the UI - startPersistedSparkUI(app).map { ui => - app.desc.appUiUrl = ui.basePath - appIdToUI(app.id) = ui - webUi.attachUI(ui) - }.getOrElse { + if (!rebuildSparkUI(app)) { // Avoid broken links if the UI is not reconstructed app.desc.appUiUrl = "" } @@ -654,30 +654,34 @@ private[spark] class Master( } /** - * Start a new SparkUI rendered from persisted storage. If this is unsuccessful for any reason, - * return None. Otherwise return the reconstructed UI. + * Rebuild a new SparkUI from the given application's event logs. + * Return whether this is successful. */ - def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { + def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name - val eventLogInfo = app.desc.eventLogInfo.getOrElse { return None } - val eventLogDir = eventLogInfo.logDir - val eventCompressionCodec = eventLogInfo.compressionCodec - val appConf = new SparkConf - eventCompressionCodec.foreach { codec => - appConf.set("spark.eventLog.compress", "true") - appConf.set("spark.io.compression.codec", codec) - } - val replayerBus = new ReplayListenerBus(appConf) - val ui = new SparkUI( - appConf, - replayerBus, - "%s (finished)".format(appName), - "/history/%s".format(app.id)) - - // Do not call ui.bind() to avoid creating a new server for each application - ui.start() - val success = replayerBus.replay(eventLogDir) - if (success) Some(ui) else None + val eventLogDir = app.desc.eventLogDir.getOrElse { return false } + val fileSystem = Utils.getHadoopFileSystem(eventLogDir) + val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) + val eventLogPaths = eventLogInfo.logPaths + val compressionCodec = eventLogInfo.compressionCodec + if (!eventLogPaths.isEmpty) { + try { + val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) + val ui = new SparkUI(replayBus, appName + " (completed)", "/history/" + app.id) + ui.start() + replayBus.replay() + app.desc.appUiUrl = ui.basePath + appIdToUI(app.id) = ui + webUi.attachUI(ui) + return true + } catch { + case t: Throwable => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), t) + } + } else { + logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) + } + false } /** Generate a new app ID given a app's submission date */ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 01d9f52f4b7b4..30c8ade408a5a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -22,8 +22,9 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging +import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.{ServerInfo, SparkUI} +import org.apache.spark.ui.SparkUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -31,7 +32,9 @@ import org.apache.spark.util.{AkkaUtils, Utils} * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { +class MasterWebUI(val master: Master, requestedPort: Int) + extends SparkUIContainer("MasterWebUI") with Logging { + val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) @@ -39,7 +42,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { private val port = requestedPort private val applicationPage = new ApplicationPage(this) private val indexPage = new IndexPage(this) - private var serverInfo: Option[ServerInfo] = None private val handlers: Seq[ServletContextHandler] = { master.masterMetricsSystem.getServletHandlers ++ @@ -57,47 +59,18 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { ) } - def bind() { + /** Bind to the HTTP server behind this web interface. */ + override def bind() { try { serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Master JettyUtils", e) + logError("Failed to create Master web UI", e) System.exit(1) } } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - - /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ - def attachUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - rootHandler.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } - } - } - - /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ - def detachUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - if (handler.isStarted) { - handler.stop() - } - rootHandler.removeHandler(handler) - } - } - - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a Master UI that was not bound to a server!") - serverInfo.get.server.stop() - } } private[spark] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 650f3da5ce3ff..5625a44549aaa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -24,7 +24,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{JettyUtils, ServerInfo, SparkUI, UIUtils} +import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -33,15 +33,14 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends Logging { + extends WebUI("WorkerWebUI") with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) private val host = Utils.localHostName() private val port = requestedPort.getOrElse( - worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) private val indexPage = new IndexPage(this) - private var serverInfo: Option[ServerInfo] = None private val handlers: Seq[ServletContextHandler] = { worker.metricsSystem.getServletHandlers ++ @@ -58,19 +57,18 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I ) } - def bind() { + /** Bind to the HTTP server behind this web interface. */ + override def bind() { try { - serverInfo = Some(JettyUtils.startJettyServer("0.0.0.0", port, handlers, worker.conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, worker.conf)) logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Worker JettyUtils", e) + logError("Failed to create Worker web UI", e) System.exit(1) } } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - private def log(request: HttpServletRequest): String = { val defaultBytes = 100 * 1024 @@ -187,13 +185,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I (startByte, endByte) } - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a Worker UI that was not bound to a server!") - serverInfo.get.server.stop() - } } private[spark] object WorkerWebUI { + val DEFAULT_PORT=8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala new file mode 100644 index 0000000000000..affda13df6531 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +/** + * A simple listener for application events. + * + * This listener expects to hear events from a single application only. If events + * from multiple applications are seen, the behavior is unspecified. + */ +private[spark] class ApplicationEventListener extends SparkListener { + var appName = "" + var sparkUser = "" + var startTime = -1L + var endTime = -1L + + def applicationStarted = startTime != -1 + + def applicationFinished = endTime != -1 + + def applicationDuration: Long = { + val difference = endTime - startTime + if (applicationStarted && applicationFinished && difference > 0) difference else -1L + } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { + appName = applicationStart.appName + startTime = applicationStart.time + sparkUser = applicationStart.sparkUser + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { + endTime = applicationEnd.time + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 217f8825c2ae9..b983c16af14f4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,11 +17,14 @@ package org.apache.spark.scheduler +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, FileLogger} +import org.apache.spark.util.{FileLogger, JsonProtocol} /** * A SparkListener that logs events to persistent storage. @@ -36,6 +39,8 @@ import org.apache.spark.util.{JsonProtocol, FileLogger} private[spark] class EventLoggingListener(appName: String, conf: SparkConf) extends SparkListener with Logging { + import EventLoggingListener._ + private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 @@ -46,17 +51,21 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) private val logger = new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) - // Information needed to replay the events logged by this listener later - val info = { - val compressionCodec = if (shouldCompress) { - Some(conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)) - } else None - EventLoggingInfo(logDir, compressionCodec) + /** + * Begin logging events. + * If compression is used, log a file that indicates which compression library is used. + */ + def start() { + logInfo("Logging events to %s".format(logDir)) + if (shouldCompress) { + val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) + logger.newFile(COMPRESSION_CODEC_PREFIX + codec) + } + logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) + logger.newFile(LOG_PREFIX + logger.fileIndex) } - logInfo("Logging events to %s".format(logDir)) - - /** Log the event as JSON */ + /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) logger.logLine(eventJson) @@ -90,9 +99,118 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logEvent(event, flushLogger = true) override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = logEvent(event, flushLogger = true) + override def onApplicationStart(event: SparkListenerApplicationStart) = + logEvent(event, flushLogger = true) + override def onApplicationEnd(event: SparkListenerApplicationEnd) = + logEvent(event, flushLogger = true) + + /** + * Stop logging events. + * In addition, create an empty special file to indicate application completion. + */ + def stop() = { + logger.newFile(APPLICATION_COMPLETE) + logger.stop() + } +} + +private[spark] object EventLoggingListener extends Logging { + val LOG_PREFIX = "EVENT_LOG_" + val SPARK_VERSION_PREFIX = "SPARK_VERSION_" + val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" + val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = new mutable.HashMap[String, CompressionCodec] + + def isEventLogFile(fileName: String): Boolean = { + fileName.startsWith(LOG_PREFIX) + } + + def isSparkVersionFile(fileName: String): Boolean = { + fileName.startsWith(SPARK_VERSION_PREFIX) + } + + def isCompressionCodecFile(fileName: String): Boolean = { + fileName.startsWith(COMPRESSION_CODEC_PREFIX) + } + + def isApplicationCompleteFile(fileName: String): Boolean = { + fileName == APPLICATION_COMPLETE + } + + def parseSparkVersion(fileName: String): String = { + if (isSparkVersionFile(fileName)) { + fileName.replaceAll(SPARK_VERSION_PREFIX, "") + } else "" + } + + def parseCompressionCodec(fileName: String): String = { + if (isCompressionCodecFile(fileName)) { + fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") + } else "" + } + + /** + * Parse the event logging information associated with the logs in the given directory. + * + * Specifically, this looks for event log files, the Spark version file, the compression + * codec file (if event logs are compressed), and the application completion file (if the + * application has run to completion). + */ + def parseLoggingInfo(logDir: Path, fileSystem: FileSystem): EventLoggingInfo = { + try { + val fileStatuses = fileSystem.listStatus(logDir) + val filePaths = + if (fileStatuses != null) { + fileStatuses.filter(!_.isDir).map(_.getPath).toSeq + } else { + Seq[Path]() + } + if (filePaths.isEmpty) { + logWarning("No files found in logging directory %s".format(logDir)) + } + EventLoggingInfo( + logPaths = filePaths.filter { path => isEventLogFile(path.getName) }, + sparkVersion = filePaths + .find { path => isSparkVersionFile(path.getName) } + .map { path => parseSparkVersion(path.getName) } + .getOrElse(""), + compressionCodec = filePaths + .find { path => isCompressionCodecFile(path.getName) } + .map { path => + val codec = EventLoggingListener.parseCompressionCodec(path.getName) + val conf = new SparkConf + conf.set("spark.io.compression.codec", codec) + codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf)) + }, + applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } + ) + } catch { + case t: Throwable => + logError("Exception in parsing logging info from directory %s".format(logDir), t) + EventLoggingInfo.empty + } + } - def stop() = logger.stop() + /** + * Parse the event logging information associated with the logs in the given directory. + */ + def parseLoggingInfo(logDir: String, fileSystem: FileSystem): EventLoggingInfo = { + parseLoggingInfo(new Path(logDir), fileSystem) + } } -// If compression is not enabled, compressionCodec is None -private[spark] case class EventLoggingInfo(logDir: String, compressionCodec: Option[String]) + +/** + * Information needed to process the event logs associated with an application. + */ +private[spark] case class EventLoggingInfo( + logPaths: Seq[Path], + sparkVersion: String, + compressionCodec: Option[CompressionCodec], + applicationComplete: Boolean = false) + +private[spark] object EventLoggingInfo { + def empty = EventLoggingInfo(Seq[Path](), "", None, applicationComplete = false) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index db76178b65501..b03665fd56d33 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io.InputStream -import java.net.URI import scala.io.Source @@ -26,63 +25,47 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.Logging import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.JsonProtocol /** - * An EventBus that replays logged events from persisted storage + * A SparkListenerBus that replays logged events from persisted storage. + * + * This class expects files to be appropriately prefixed as specified in EventLoggingListener. + * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. */ -private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus with Logging { - private val compressed = conf.getBoolean("spark.eventLog.compress", false) +private[spark] class ReplayListenerBus( + logPaths: Seq[Path], + fileSystem: FileSystem, + compressionCodec: Option[CompressionCodec]) + extends SparkListenerBus with Logging { - // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(conf) + private var replayed = false - /** - * Return a list of paths representing log files in the given directory. - */ - private def getLogFilePaths(logDir: String, fileSystem: FileSystem): Array[Path] = { - val path = new Path(logDir) - if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { - logWarning("Log path provided is not a valid directory: %s".format(logDir)) - return Array[Path]() - } - val logStatus = fileSystem.listStatus(path) - if (logStatus == null || !logStatus.exists(!_.isDir)) { - logWarning("Log path provided contains no log files: %s".format(logDir)) - return Array[Path]() - } - logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + if (logPaths.length == 0) { + logWarning("Log path provided contains no log files.") } /** * Replay each event in the order maintained in the given logs. + * This should only be called exactly once. */ - def replay(logDir: String): Boolean = { - val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - val logPaths = getLogFilePaths(logDir, fileSystem) - if (logPaths.length == 0) { - return false - } - + def replay() { + assert(!replayed, "ReplayListenerBus cannot replay events more than once") logPaths.foreach { path => // Keep track of input streams at all levels to close them later // This is necessary because an exception can occur in between stream initializations var fileStream: Option[InputStream] = None var bufferedStream: Option[InputStream] = None var compressStream: Option[InputStream] = None - var currentLine = "" + var currentLine = "" try { - currentLine = "" fileStream = Some(fileSystem.open(path)) bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) - compressStream = - if (compressed) { - Some(compressionCodec.compressedInputStream(bufferedStream.get)) - } else bufferedStream + compressStream = Some(wrapForCompression(bufferedStream.get)) - // Parse each line as an event and post it to all attached listeners + // Parse each line as an event and post the event to all attached listeners val lines = Source.fromInputStream(compressStream.get).getLines() lines.foreach { line => currentLine = line @@ -98,7 +81,11 @@ private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus compressStream.foreach(_.close()) } } - fileSystem.close() - true + replayed = true + } + + /** If a compression codec is specified, wrap the given stream in a compression stream. */ + private def wrapForCompression(stream: InputStream): InputStream = { + compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index ced20350d5356..378cf1aaebe7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -75,6 +75,11 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) + extends SparkListenerEvent + +case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent + /** An event used in the listener to shutdown the listener daemon thread. */ private[spark] case object SparkListenerShutdown extends SparkListenerEvent @@ -141,6 +146,16 @@ trait SparkListener { * Called when an RDD is manually unpersisted by the application */ def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } + + /** + * Called when the application starts + */ + def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } + + /** + * Called when the application ends + */ + def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 729e120497571..d6df193d9bcf8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -61,6 +61,10 @@ private[spark] trait SparkListenerBus { sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) case unpersistRDD: SparkListenerUnpersistRDD => sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + case applicationStart: SparkListenerApplicationStart => + sparkListeners.foreach(_.onApplicationStart(applicationStart)) + case applicationEnd: SparkListenerApplicationEnd => + sparkListeners.foreach(_.onApplicationEnd(applicationEnd)) case SparkListenerShutdown => } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 25b7472a99cdb..936e9db80573d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -49,7 +49,7 @@ private[spark] class SparkDeploySchedulerBackend( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.eventLoggingInfo) + sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala index 555486830a769..132502b75f8cd 100644 --- a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala @@ -23,6 +23,6 @@ import java.io.File * References a particular segment of a file (potentially the entire file), * based off an offset and a length. */ -private[spark] class FileSegment(val file: File, val offset: Long, val length : Long) { +private[spark] class FileSegment(val file: File, val offset: Long, val length: Long) { override def toString = "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length) } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index f53df7fbedf39..b8e6e15880bf5 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -34,23 +34,22 @@ private[spark] class SparkUI( val sc: SparkContext, conf: SparkConf, val listenerBus: SparkListenerBus, - val appName: String, + var appName: String, val basePath: String = "") - extends Logging { + extends WebUI("SparkUI") with Logging { def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) - def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = - this(null, conf, listenerBus, appName, basePath) + def this(listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, new SparkConf, listenerBus, appName, basePath) // If SparkContext is not provided, assume the associated application is not live val live = sc != null val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - private val bindHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) - private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt - private var serverInfo: Option[ServerInfo] = None + private val localHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) + private val port = conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) private val storage = new BlockManagerUI(this) private val jobs = new JobProgressUI(this) @@ -77,20 +76,10 @@ private[spark] class SparkUI( // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener - /** Bind the HTTP server which backs this web interface */ - def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) - logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Spark JettyUtils", e) - System.exit(1) - } + def setAppName(name: String) { + appName = name } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - /** Initialize all components of the server */ def start() { storage.start() @@ -106,9 +95,21 @@ private[spark] class SparkUI( listenerBus.addListener(exec.listener) } - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a SparkUI that was not bound to a server!") - serverInfo.get.server.stop() + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) + logInfo("Started Spark web UI at http://%s:%d".format(publicHost, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Spark web UI", e) + System.exit(1) + } + } + + /** Stop the server behind this web interface. Only valid after bind(). */ + override def stop() { + super.stop() logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } @@ -117,6 +118,6 @@ private[spark] class SparkUI( } private[spark] object SparkUI { - val DEFAULT_PORT = "4040" + val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index a7b872f3445a4..2cc7582eca8a3 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -20,6 +20,25 @@ package org.apache.spark.ui import java.text.SimpleDateFormat import java.util.Date +private[spark] abstract class WebUI(name: String) { + protected var serverInfo: Option[ServerInfo] = None + + /** + * Bind to the HTTP server behind this web interface. + * Overridden implementation should set serverInfo. + */ + def bind() { } + + /** Return the actual port to which this server is bound. Only valid after bind(). */ + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + + /** Stop the server behind this web interface. Only valid after bind(). */ + def stop() { + assert(serverInfo.isDefined, "Attempted to stop %s before binding to a server!".format(name)) + serverInfo.get.server.stop() + } +} + /** * Utilities used throughout the web UI. */ @@ -45,6 +64,6 @@ private[spark] object WebUI { return "%.0f min".format(minutes) } val hours = minutes / 60 - return "%.1f h".format(hours) + "%.1f h".format(hours) } } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 23e90c34d5b33..33df97187ea78 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -29,10 +29,11 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Environment private[ui] class EnvironmentUI(parent: SparkUI) { - private val appName = parent.appName private val basePath = parent.basePath private var _listener: Option[EnvironmentListener] = None + private def appName = parent.appName + lazy val listener = _listener.get def start() { diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 031ed88a493a8..77a38a1d3aa7c 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -33,10 +33,11 @@ import org.apache.spark.ui.{SparkUI, UIUtils} import org.apache.spark.util.Utils private[ui] class ExecutorsUI(parent: SparkUI) { - private val appName = parent.appName private val basePath = parent.basePath private var _listener: Option[ExecutorsListener] = None + private def appName = parent.appName + lazy val listener = _listener.get def start() { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 70d62b66a4829..f811aff616bcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -27,13 +27,14 @@ import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class IndexPage(parent: JobProgressUI) { - private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index b2c67381cc3da..ad1a12cdc4e36 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -29,7 +29,6 @@ import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobProgressUI(parent: SparkUI) { - val appName = parent.appName val basePath = parent.basePath val live = parent.live val sc = parent.sc @@ -42,6 +41,8 @@ private[ui] class JobProgressUI(parent: SparkUI) { private val poolPage = new PoolPage(this) private var _listener: Option[JobProgressListener] = None + def appName = parent.appName + def start() { val conf = if (live) sc.conf else new SparkConf _listener = Some(new JobProgressListener(conf)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index bd33182b70059..3638e6035ba81 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -27,12 +27,13 @@ import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressUI) { - private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val poolName = request.getParameter("poolname") diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0c55f2ee7e944..0bcbd7461cc5b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -28,10 +28,11 @@ import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index a7b24ff695214..16996a2da1e72 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -30,7 +30,6 @@ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class BlockManagerUI(parent: SparkUI) { - val appName = parent.appName val basePath = parent.basePath private val indexPage = new IndexPage(this) @@ -39,6 +38,8 @@ private[ui] class BlockManagerUI(parent: SparkUI) { lazy val listener = _listener.get + def appName = parent.appName + def start() { _listener = Some(new BlockManagerListener(parent.storageStatusListener)) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 0fa461e5e9d27..4f6acc30a88c4 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -28,10 +28,11 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[ui] class IndexPage(parent: BlockManagerUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 3f42eba4ece00..75ee9976d7b5f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -28,10 +28,11 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ private[ui] class RDDPage(parent: BlockManagerUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt val storageStatusList = listener.storageStatusList diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index b5f2ec6831d26..0080a8b342b05 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -49,7 +49,7 @@ private[spark] class FileLogger( } private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - private var fileIndex = 0 + var fileIndex = 0 // Only used if compression is enabled private lazy val compressionCodec = CompressionCodec.createCodec(conf) @@ -57,10 +57,9 @@ private[spark] class FileLogger( // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None - private var writer: Option[PrintWriter] = { - createLogDir() - Some(createWriter()) - } + private var writer: Option[PrintWriter] = None + + createLogDir() /** * Create a logging directory with the given path. @@ -84,8 +83,8 @@ private[spark] class FileLogger( /** * Create a new writer for the file identified by the given path. */ - private def createWriter(): PrintWriter = { - val logPath = logDir + "/" + fileIndex + private def createWriter(fileName: String): PrintWriter = { + val logPath = logDir + "/" + fileName val uri = new URI(logPath) /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). @@ -147,13 +146,17 @@ private[spark] class FileLogger( } /** - * Start a writer for a new file if one does not already exit. + * Start a writer for a new file, closing the existing one if it exists. + * @param fileName Name of the new file, defaulting to the file index if not provided. */ - def start() { - writer.getOrElse { - fileIndex += 1 - writer = Some(createWriter()) + def newFile(fileName: String = "") { + fileIndex += 1 + writer.foreach(_.close()) + val name = fileName match { + case "" => fileIndex.toString + case _ => fileName } + writer = Some(createWriter(name)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 19654892bf661..d990fd49ef834 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -62,6 +62,10 @@ private[spark] object JsonProtocol { blockManagerRemovedToJson(blockManagerRemoved) case unpersistRDD: SparkListenerUnpersistRDD => unpersistRDDToJson(unpersistRDD) + case applicationStart: SparkListenerApplicationStart => + applicationStartToJson(applicationStart) + case applicationEnd: SparkListenerApplicationEnd => + applicationEndToJson(applicationEnd) // Not used, but keeps compiler happy case SparkListenerShutdown => JNothing @@ -157,6 +161,18 @@ private[spark] object JsonProtocol { ("RDD ID" -> unpersistRDD.rddId) } + def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("App Name" -> applicationStart.appName) ~ + ("Timestamp" -> applicationStart.time) ~ + ("User" -> applicationStart.sparkUser) + } + + def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationEnd)) ~ + ("Timestamp" -> applicationEnd.time) + } + /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | @@ -346,6 +362,8 @@ private[spark] object JsonProtocol { val blockManagerAdded = Utils.getFormattedClassName(SparkListenerBlockManagerAdded) val blockManagerRemoved = Utils.getFormattedClassName(SparkListenerBlockManagerRemoved) val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) + val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) + val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -359,6 +377,8 @@ private[spark] object JsonProtocol { case `blockManagerAdded` => blockManagerAddedFromJson(json) case `blockManagerRemoved` => blockManagerRemovedFromJson(json) case `unpersistRDD` => unpersistRDDFromJson(json) + case `applicationStart` => applicationStartFromJson(json) + case `applicationEnd` => applicationEndFromJson(json) } } @@ -430,6 +450,17 @@ private[spark] object JsonProtocol { SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) } + def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { + val appName = (json \ "App Name").extract[String] + val time = (json \ "Timestamp").extract[Long] + val sparkUser = (json \ "User").extract[String] + SparkListenerApplicationStart(appName, time, sparkUser) + } + + def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { + SparkListenerApplicationEnd((json \ "Timestamp").extract[Long]) + } + /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 59da51f3e0297..166f48ce7342e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,7 +26,6 @@ import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.SortedSet import scala.io.Source import scala.reflect.ClassTag @@ -1022,4 +1021,11 @@ private[spark] object Utils extends Logging { def getHadoopFileSystem(path: URI): FileSystem = { FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) } + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + */ + def getHadoopFileSystem(path: String): FileSystem = { + getHadoopFileSystem(new URI(path)) + } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index beac656f573b4..8c06a2d9aa4ab 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, Success} +import org.apache.spark.{LocalSparkContext, SparkConf, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 0342a8aff3c28..f75297a02dc8b 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.util.{Properties, UUID} +import java.util.Properties import scala.collection.Map @@ -52,6 +52,8 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved( BlockManagerId("Scarce", "to be counted...", 100, 200)) val unpersistRdd = SparkListenerUnpersistRDD(12345) + val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") + val applicationEnd = SparkListenerApplicationEnd(42L) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -64,6 +66,8 @@ class JsonProtocolSuite extends FunSuite { testEvent(blockManagerAdded, blockManagerAddedJsonString) testEvent(blockManagerRemoved, blockManagerRemovedJsonString) testEvent(unpersistRdd, unpersistRDDJsonString) + testEvent(applicationStart, applicationStartJsonString) + testEvent(applicationEnd, applicationEndJsonString) } test("Dependent Classes") { @@ -208,7 +212,13 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => - assert(e1.rddId === e2.rddId) + assert(e1.rddId == e2.rddId) + case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => + assert(e1.appName == e2.appName) + assert(e1.time == e2.time) + assert(e1.sparkUser == e2.sparkUser) + case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => + assert(e1.time == e2.time) case (SparkListenerShutdown, SparkListenerShutdown) => case _ => fail("Events don't match in types!") } @@ -553,4 +563,14 @@ class JsonProtocolSuite extends FunSuite { {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} """ + private val applicationStartJsonString = + """ + {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42, + "User":"Garfield"} + """ + + private val applicationEndJsonString = + """ + {"Event":"SparkListenerApplicationEnd","Timestamp":42} + """ } diff --git a/docs/monitoring.md b/docs/monitoring.md index 15bfb041780da..4c91c3a5929bf 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -12,17 +12,77 @@ displays useful information about the application. This includes: * A list of scheduler stages and tasks * A summary of RDD sizes and memory usage -* Information about the running executors * Environmental information. +* Information about the running executors You can access this interface by simply opening `http://:4040` in a web browser. -If multiple SparkContexts are running on the same host, they will bind to succesive ports +If multiple SparkContexts are running on the same host, they will bind to successive ports beginning with 4040 (4041, 4042, etc). -Spark's Standalone Mode cluster manager also has its own -[web UI](spark-standalone.html#monitoring-and-logging). +Note that this information is only available for the duration of the application by default. +To view the web UI after the fact, set `spark.eventLog.enabled` to true before starting the +application. This configures Spark to log Spark events that encode the information displayed +in the UI to persisted storage. -Note that in both of these UIs, the tables are sortable by clicking their headers, +## Viewing After the Fact + +Spark's Standalone Mode cluster manager also has its own +[web UI](spark-standalone.html#monitoring-and-logging). If an application has logged events over +the course of its lifetime, then the Standalone master's web UI will automatically re-render the +application's UI after the application has finished. + +If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of a finished +application through Spark's history server, provided that the application's event logs exist. +You can start a the history server by executing: + + ./sbin/start-history-server.sh + +The base logging directory must be supplied, and should contain sub-directories that each +represents an application's event logs. This creates a web interface at +`http://:18080` by default. The history server depends on the following variables: + + + + + + + + + + + +
Environment VariableMeaning
SPARK_DAEMON_MEMORYMemory to allocate to the history server. (default: 512m).
SPARK_DAEMON_JAVA_OPTSJVM options for the history server (default: none).
+ +Further, the history server can be configured as follows: + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.history.updateInterval10 + The period, in seconds, at which information displayed by this history server is updated. + Each update checks for any changes made to the event logs in persisted storage. +
spark.history.retainedApplications250 + The number of application UIs to retain. If this cap is exceeded, then the oldest + applications will be removed. +
spark.history.ui.port18080 + The port to which the web interface of the history server binds. +
+ +Note that in all of these UIs, the tables are sortable by clicking their headers, making it easy to identify slow tasks, data skew, etc. # Metrics diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 3ebf288130fb6..910b31d209e13 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -116,14 +116,14 @@ trait SparkILoopInit { } } - def initializeSpark() { + def initializeSpark() { intp.beQuietDuring { command(""" @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext(); """) command("import org.apache.spark.SparkContext._") } - echo("Spark context available as sc.") + echo("Spark context available as sc.") } // code to be executed only after the interpreter is initialized diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh new file mode 100755 index 0000000000000..4a90c68763b68 --- /dev/null +++ b/sbin/start-history-server.sh @@ -0,0 +1,37 @@ +#!/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 history server on the machine this script is executed on. +# +# Usage: start-history-server.sh [] +# Example: ./start-history-server.sh --dir /tmp/spark-events --port 18080 +# + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +if [ $# -lt 1 ]; then + echo "Usage: ./start-history-server.sh " + echo "Example: ./start-history-server.sh /tmp/spark-events" + exit +fi + +LOG_DIR=$1 + +"$sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 --dir "$LOG_DIR" diff --git a/sbin/stop-history-server.sh b/sbin/stop-history-server.sh new file mode 100755 index 0000000000000..c0034ad641cbe --- /dev/null +++ b/sbin/stop-history-server.sh @@ -0,0 +1,25 @@ +#!/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. +# + +# Stops the history server on the machine this script is executed on. + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.history.HistoryServer 1 From 3bd312940e2f5250edaf3e88d6c23de25bb1d0a9 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 10 Apr 2014 11:17:41 -0700 Subject: [PATCH 04/61] SPARK-1428: MLlib should convert non-float64 NumPy arrays to float64 instead of complaining Author: Sandeep Closes #356 from techaddict/1428 and squashes the following commits: 3bdf5f6 [Sandeep] SPARK-1428: MLlib should convert non-float64 NumPy arrays to float64 instead of complaining --- python/pyspark/mllib/_common.py | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index 20a0e309d1494..7ef251d24c77e 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -15,8 +15,9 @@ # limitations under the License. # -from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape +from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape, complex, issubdtype from pyspark import SparkContext, RDD +import numpy as np from pyspark.serializers import Serializer import struct @@ -47,13 +48,22 @@ def _deserialize_byte_array(shape, ba, offset): return ar.copy() def _serialize_double_vector(v): - """Serialize a double vector into a mutually understood format.""" + """Serialize a double vector into a mutually understood format. + + >>> x = array([1,2,3]) + >>> y = _deserialize_double_vector(_serialize_double_vector(x)) + >>> array_equal(y, array([1.0, 2.0, 3.0])) + True + """ if type(v) != ndarray: raise TypeError("_serialize_double_vector called on a %s; " "wanted ndarray" % type(v)) + """complex is only datatype that can't be converted to float64""" + if issubdtype(v.dtype, complex): + raise TypeError("_serialize_double_vector called on a %s; " + "wanted ndarray" % type(v)) if v.dtype != float64: - raise TypeError("_serialize_double_vector called on an ndarray of %s; " - "wanted ndarray of float64" % v.dtype) + v = v.astype(float64) if v.ndim != 1: raise TypeError("_serialize_double_vector called on a %ddarray; " "wanted a 1darray" % v.ndim) From 7b52b66312994d4dbf243eadb6d27eb06350a81f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 10 Apr 2014 14:43:29 -0700 Subject: [PATCH 05/61] Revert "SPARK-1433: Upgrade Mesos dependency to 0.17.0" This reverts commit 12c077d5aa0b76a808a55db625c9677a52bd43f9. --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 6 ++---- .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- docs/_config.yml | 2 +- pom.xml | 6 +++--- project/SparkBuild.scala | 2 +- 5 files changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index c478e685641d7..06b041e1fd9a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -194,12 +194,10 @@ private[spark] class CoarseMesosSchedulerBackend( .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", sc.executorMemory)) .build() - d.launchTasks(Collections.singletonList(offer.getId), - Collections.singletonList(task), - filters) + d.launchTasks(offer.getId, Collections.singletonList(task), filters) } else { // Filter it out - d.declineOffer(offer.getId, filters) + d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index f878ae338fc95..dfdcafe19fb93 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -223,7 +223,7 @@ private[spark] class MesosSchedulerBackend( // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? for (i <- 0 until offers.size) { - d.launchTasks(Collections.singletonList(offers(i).getId), mesosTasks(i), filters) + d.launchTasks(offers(i).getId, mesosTasks(i), filters) } } } finally { diff --git a/docs/_config.yml b/docs/_config.yml index bd5ed6c9220d2..d585b8c5ea763 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,6 +7,6 @@ SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.17.0 +MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/pom.xml b/pom.xml index 11511bcb9da52..c03bb35c99442 100644 --- a/pom.xml +++ b/pom.xml @@ -112,7 +112,7 @@ 2.10.4 2.10 - 0.17.0 + 0.13.0 org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -848,7 +848,7 @@ - + hadoop-provided @@ -893,6 +893,6 @@ - + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 694f90a83ab67..21163760e6277 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -333,7 +333,7 @@ object SparkBuild extends Build { "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.17.0", + "org.apache.mesos" % "mesos" % "0.13.0", "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From f0466625200842f3cc486e9aa1caa417586be533 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 10 Apr 2014 14:59:58 -0700 Subject: [PATCH 06/61] Update tuning.md http://stackoverflow.com/questions/9699071/what-is-the-javas-internal-represention-for-string-modified-utf-8-utf-16 Author: Andrew Ash Closes #384 from ash211/patch-2 and squashes the following commits: da1b0be [Andrew Ash] Update tuning.md --- docs/tuning.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/tuning.md b/docs/tuning.md index 093df3187a789..cc069f0e84b9c 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -90,9 +90,10 @@ than the "raw" data inside their fields. This is due to several reasons: * Each distinct Java object has an "object header", which is about 16 bytes and contains information such as a pointer to its class. For an object with very little data in it (say one `Int` field), this can be bigger than the data. -* Java Strings have about 40 bytes of overhead over the raw string data (since they store it in an +* Java `String`s have about 40 bytes of overhead over the raw string data (since they store it in an array of `Char`s and keep extra data such as the length), and store each character - as *two* bytes due to Unicode. Thus a 10-character string can easily consume 60 bytes. + as *two* bytes due to `String`'s internal usage of UTF-16 encoding. Thus a 10-character string can + easily consume 60 bytes. * Common collection classes, such as `HashMap` and `LinkedList`, use linked data structures, where there is a "wrapper" object for each entry (e.g. `Map.Entry`). This object not only has a header, but also pointers (typically 8 bytes each) to the next object in the list. From 930b70f0523e96fe01c1317ef7fad1b76b36d4d9 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 10 Apr 2014 15:04:13 -0700 Subject: [PATCH 07/61] Remove Unnecessary Whitespace's stack these together in a commit else they show up chunk by chunk in different commits. Author: Sandeep Closes #380 from techaddict/white_space and squashes the following commits: b58f294 [Sandeep] Remove Unnecessary Whitespace's --- .../org/apache/spark/bagel/BagelSuite.scala | 4 +- .../api/java/function/FlatMapFunction.java | 2 +- .../api/java/function/FlatMapFunction2.java | 2 +- .../org/apache/spark/HttpFileServer.scala | 14 +- .../scala/org/apache/spark/HttpServer.scala | 6 +- .../scala/org/apache/spark/Partition.scala | 2 +- .../org/apache/spark/SecurityManager.scala | 88 +++---- .../org/apache/spark/SparkException.scala | 2 +- .../org/apache/spark/SparkHadoopWriter.scala | 20 +- .../org/apache/spark/SparkSaslClient.scala | 10 +- .../org/apache/spark/SparkSaslServer.scala | 6 +- .../scala/org/apache/spark/TestUtils.scala | 2 +- .../spark/broadcast/TorrentBroadcast.scala | 2 +- .../apache/spark/deploy/ClientArguments.scala | 2 +- .../spark/deploy/worker/WorkerArguments.scala | 8 +- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../spark/executor/ExecutorExitCode.scala | 8 +- .../executor/ExecutorURLClassLoader.scala | 2 +- .../apache/spark/metrics/sink/CsvSink.scala | 2 +- .../org/apache/spark/network/Connection.scala | 8 +- .../apache/spark/network/ConnectionId.scala | 6 +- .../spark/network/ConnectionManager.scala | 28 +-- .../spark/network/ConnectionManagerTest.scala | 24 +- .../apache/spark/network/ReceiverTest.scala | 2 +- .../spark/network/SecurityMessage.scala | 48 ++-- .../spark/network/netty/FileHeader.scala | 4 +- .../apache/spark/partial/PartialResult.scala | 4 +- .../apache/spark/rdd/DoubleRDDFunctions.scala | 8 +- .../spark/rdd/PartitionerAwareUnionRDD.scala | 2 +- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../spark/scheduler/LiveListenerBus.scala | 214 +++++++++--------- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 4 +- .../apache/spark/storage/BlockMessage.scala | 22 +- .../spark/storage/BlockMessageArray.scala | 26 +-- .../org/apache/spark/ui/JettyUtils.scala | 4 +- .../scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../apache/spark/util/ClosureCleaner.scala | 22 +- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../org/apache/spark/util/NextIterator.scala | 4 +- .../org/apache/spark/util/StatCounter.scala | 4 +- .../scala/org/apache/spark/util/Vector.scala | 2 +- .../spark/util/random/XORShiftRandom.scala | 16 +- .../org/apache/spark/AkkaUtilsSuite.scala | 20 +- .../scala/org/apache/spark/DriverSuite.scala | 2 +- .../org/apache/spark/FileServerSuite.scala | 4 +- .../scala/org/apache/spark/FileSuite.scala | 2 +- .../deploy/worker/WorkerWatcherSuite.scala | 2 +- .../WholeTextFileRecordReaderSuite.scala | 2 +- .../rdd/ParallelCollectionSplitSuite.scala | 26 +-- .../spark/scheduler/SparkListenerSuite.scala | 4 +- .../scheduler/TaskSchedulerImplSuite.scala | 2 +- .../scala/org/apache/spark/ui/UISuite.scala | 4 +- .../spark/util/ClosureCleanerSuite.scala | 2 +- .../apache/spark/util/NextIteratorSuite.scala | 4 +- .../util/random/XORShiftRandomSuite.scala | 20 +- .../streaming/mqtt/MQTTInputDStream.scala | 16 +- .../twitter/TwitterInputDStream.scala | 6 +- .../org/apache/spark/graphx/GraphOps.scala | 2 +- .../apache/spark/graphx/GraphOpsSuite.scala | 2 +- .../spark/mllib/optimization/Optimizer.scala | 2 +- .../GeneralizedLinearAlgorithm.scala | 4 +- .../spark/repl/ExecutorClassLoader.scala | 4 +- .../org/apache/spark/repl/SparkImports.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 6 +- .../sql/catalyst/expressions/Expression.scala | 12 +- .../expressions/stringOperations.scala | 28 +-- .../spark/sql/catalyst/types/dataTypes.scala | 4 +- .../ExpressionEvaluationSuite.scala | 10 +- .../sql/ScalaReflectionRelationSuite.scala | 2 +- .../apache/spark/streaming/Checkpoint.scala | 14 +- .../org/apache/spark/streaming/Interval.scala | 8 +- .../org/apache/spark/streaming/Time.scala | 4 +- .../dstream/DStreamCheckpointData.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 2 +- .../streaming/dstream/QueueInputDStream.scala | 8 +- .../streaming/receivers/ActorReceiver.scala | 2 +- .../apache/spark/streaming/util/Clock.scala | 26 +-- .../spark/streaming/util/RawTextHelper.scala | 18 +- .../spark/streaming/util/RecurringTimer.scala | 8 +- .../apache/spark/streaming/JavaAPISuite.java | 2 +- 82 files changed, 467 insertions(+), 467 deletions(-) diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 9c37fadb78d2f..69144e3e657bf 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -28,9 +28,9 @@ class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializ class TestMessage(val targetId: String) extends Message[String] with Serializable class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeouts { - + var sc: SparkContext = _ - + after { if (sc != null) { sc.stop() diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java index fa75842047c6a..23f5fdd43631b 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java @@ -24,4 +24,4 @@ */ public interface FlatMapFunction extends Serializable { public Iterable call(T t) throws Exception; -} \ No newline at end of file +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java index d1fdec072443d..c48e92f535ff5 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java @@ -24,4 +24,4 @@ */ public interface FlatMapFunction2 extends Serializable { public Iterable call(T1 t1, T2 t2) throws Exception; -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 3d7692ea8a49e..a6e300d345786 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -24,13 +24,13 @@ import com.google.common.io.Files import org.apache.spark.util.Utils private[spark] class HttpFileServer(securityManager: SecurityManager) extends Logging { - + var baseDir : File = null var fileDir : File = null var jarDir : File = null var httpServer : HttpServer = null var serverUri : String = null - + def initialize() { baseDir = Utils.createTempDir() fileDir = new File(baseDir, "files") @@ -43,24 +43,24 @@ private[spark] class HttpFileServer(securityManager: SecurityManager) extends Lo serverUri = httpServer.uri logDebug("HTTP file server started at: " + serverUri) } - + def stop() { httpServer.stop() } - + def addFile(file: File) : String = { addFileToDir(file, fileDir) serverUri + "/files/" + file.getName } - + def addJar(file: File) : String = { addFileToDir(file, jarDir) serverUri + "/jars/" + file.getName } - + def addFileToDir(file: File, dir: File) : String = { Files.copy(file, new File(dir, file.getName)) dir + "/" + file.getName } - + } diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index cb5df25fa48df..7e9b517f901a2 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -83,19 +83,19 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan } } - /** + /** * Setup Jetty to the HashLoginService using a single user with our * shared secret. Configure it to use DIGEST-MD5 authentication so that the password * isn't passed in plaintext. */ private def setupSecurityHandler(securityMgr: SecurityManager): ConstraintSecurityHandler = { val constraint = new Constraint() - // use DIGEST-MD5 as the authentication mechanism + // use DIGEST-MD5 as the authentication mechanism constraint.setName(Constraint.__DIGEST_AUTH) constraint.setRoles(Array("user")) constraint.setAuthenticate(true) constraint.setDataConstraint(Constraint.DC_NONE) - + val cm = new ConstraintMapping() cm.setConstraint(constraint) cm.setPathSpec("/*") diff --git a/core/src/main/scala/org/apache/spark/Partition.scala b/core/src/main/scala/org/apache/spark/Partition.scala index 87914a061f5d7..27892dbd2a0bc 100644 --- a/core/src/main/scala/org/apache/spark/Partition.scala +++ b/core/src/main/scala/org/apache/spark/Partition.scala @@ -25,7 +25,7 @@ trait Partition extends Serializable { * Get the split's index within its parent RDD */ def index: Int - + // A better default implementation of HashCode override def hashCode(): Int = index } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 2237ee3bb7aad..b52f2d4f416b2 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -25,93 +25,93 @@ import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil -/** - * Spark class responsible for security. - * +/** + * Spark class responsible for security. + * * In general this class should be instantiated by the SparkEnv and most components - * should access it from that. There are some cases where the SparkEnv hasn't been + * should access it from that. There are some cases where the SparkEnv hasn't been * initialized yet and this class must be instantiated directly. - * + * * Spark currently supports authentication via a shared secret. * Authentication can be configured to be on via the 'spark.authenticate' configuration - * parameter. This parameter controls whether the Spark communication protocols do + * parameter. This parameter controls whether the Spark communication protocols do * authentication using the shared secret. This authentication is a basic handshake to * make sure both sides have the same shared secret and are allowed to communicate. - * If the shared secret is not identical they will not be allowed to communicate. - * - * The Spark UI can also be secured by using javax servlet filters. A user may want to - * secure the UI if it has data that other users should not be allowed to see. The javax - * servlet filter specified by the user can authenticate the user and then once the user - * is logged in, Spark can compare that user versus the view acls to make sure they are - * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' + * If the shared secret is not identical they will not be allowed to communicate. + * + * The Spark UI can also be secured by using javax servlet filters. A user may want to + * secure the UI if it has data that other users should not be allowed to see. The javax + * servlet filter specified by the user can authenticate the user and then once the user + * is logged in, Spark can compare that user versus the view acls to make sure they are + * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' * control the behavior of the acls. Note that the person who started the application * always has view access to the UI. * * Spark does not currently support encryption after authentication. - * + * * At this point spark has multiple communication protocols that need to be secured and * different underlying mechanisms are used depending on the protocol: * - * - Akka -> The only option here is to use the Akka Remote secure-cookie functionality. - * Akka remoting allows you to specify a secure cookie that will be exchanged - * and ensured to be identical in the connection handshake between the client - * and the server. If they are not identical then the client will be refused - * to connect to the server. There is no control of the underlying - * authentication mechanism so its not clear if the password is passed in + * - Akka -> The only option here is to use the Akka Remote secure-cookie functionality. + * Akka remoting allows you to specify a secure cookie that will be exchanged + * and ensured to be identical in the connection handshake between the client + * and the server. If they are not identical then the client will be refused + * to connect to the server. There is no control of the underlying + * authentication mechanism so its not clear if the password is passed in * plaintext or uses DIGEST-MD5 or some other mechanism. * Akka also has an option to turn on SSL, this option is not currently supported * but we could add a configuration option in the future. - * - * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty - * for the HttpServer. Jetty supports multiple authentication mechanisms - - * Basic, Digest, Form, Spengo, etc. It also supports multiple different login + * + * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty + * for the HttpServer. Jetty supports multiple authentication mechanisms - + * Basic, Digest, Form, Spengo, etc. It also supports multiple different login * services - Hash, JAAS, Spnego, JDBC, etc. Spark currently uses the HashLoginService - * to authenticate using DIGEST-MD5 via a single user and the shared secret. + * to authenticate using DIGEST-MD5 via a single user and the shared secret. * Since we are using DIGEST-MD5, the shared secret is not passed on the wire * in plaintext. * We currently do not support SSL (https), but Jetty can be configured to use it * so we could add a configuration option for this in the future. - * + * * The Spark HttpServer installs the HashLoginServer and configures it to DIGEST-MD5. - * Any clients must specify the user and password. There is a default + * Any clients must specify the user and password. There is a default * Authenticator installed in the SecurityManager to how it does the authentication * and in this case gets the user name and password from the request. * - * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously - * exchange messages. For this we use the Java SASL - * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 + * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously + * exchange messages. For this we use the Java SASL + * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 * as the authentication mechanism. This means the shared secret is not passed * over the wire in plaintext. * Note that SASL is pluggable as to what mechanism it uses. We currently use * DIGEST-MD5 but this could be changed to use Kerberos or other in the future. * Spark currently supports "auth" for the quality of protection, which means * the connection is not supporting integrity or privacy protection (encryption) - * after authentication. SASL also supports "auth-int" and "auth-conf" which + * after authentication. SASL also supports "auth-int" and "auth-conf" which * SPARK could be support in the future to allow the user to specify the quality - * of protection they want. If we support those, the messages will also have to + * of protection they want. If we support those, the messages will also have to * be wrapped and unwrapped via the SaslServer/SaslClient.wrap/unwrap API's. - * - * Since the connectionManager does asynchronous messages passing, the SASL + * + * Since the connectionManager does asynchronous messages passing, the SASL * authentication is a bit more complex. A ConnectionManager can be both a client * and a Server, so for a particular connection is has to determine what to do. - * A ConnectionId was added to be able to track connections and is used to + * A ConnectionId was added to be able to track connections and is used to * match up incoming messages with connections waiting for authentication. * If its acting as a client and trying to send a message to another ConnectionManager, * it blocks the thread calling sendMessage until the SASL negotiation has occurred. * The ConnectionManager tracks all the sendingConnections using the ConnectionId * and waits for the response from the server and does the handshake. * - * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters + * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters * can be used. Yarn requires a specific AmIpFilter be installed for security to work * properly. For non-Yarn deployments, users can write a filter to go through a * companies normal login service. If an authentication filter is in place then the * SparkUI can be configured to check the logged in user against the list of users who * have view acls to see if that user is authorized. - * The filters can also be used for many different purposes. For instance filters + * The filters can also be used for many different purposes. For instance filters * could be used for logging, encryption, or compression. - * + * * The exact mechanisms used to generate/distributed the shared secret is deployment specific. - * + * * For Yarn deployments, the secret is automatically generated using the Akka remote * Crypt.generateSecureCookie() API. The secret is placed in the Hadoop UGI which gets passed * around via the Hadoop RPC mechanism. Hadoop RPC can be configured to support different levels @@ -121,7 +121,7 @@ import org.apache.spark.deploy.SparkHadoopUtil * to reduce the possibility of web based attacks through YARN. Hadoop can be configured to use * filters to do authentication. That authentication then happens via the ResourceManager Proxy * and Spark will use that to do authorization against the view acls. - * + * * For other Spark deployments, the shared secret must be specified via the * spark.authenticate.secret config. * All the nodes (Master and Workers) and the applications need to have the same shared secret. @@ -152,7 +152,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { " are ui acls enabled: " + uiAclsOn + " users with view permissions: " + viewAcls.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. - // This is needed by the HTTP client fetching from the HttpServer. Put here so its + // This is needed by the HTTP client fetching from the HttpServer. Put here so its // only set once. if (authOn) { Authenticator.setDefault( @@ -214,12 +214,12 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { def uiAclsEnabled(): Boolean = uiAclsOn /** - * Checks the given user against the view acl list to see if they have + * Checks the given user against the view acl list to see if they have * authorization to view the UI. If the UI acls must are disabled * via spark.ui.acls.enable, all users have view access. - * + * * @param user to see if is authorized - * @return true is the user has permission, otherwise false + * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index d34e47e8cac22..4351ed74b67fc 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -20,5 +20,5 @@ package org.apache.spark class SparkException(message: String, cause: Throwable) extends Exception(message, cause) { - def this(message: String) = this(message, null) + def this(message: String) = this(message, null) } diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index b92ea01a877f7..f6703986bdf11 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -42,7 +42,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) private val now = new Date() private val conf = new SerializableWritable(jobConf) - + private var jobID = 0 private var splitID = 0 private var attemptID = 0 @@ -58,8 +58,8 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def preSetup() { setIDs(0, 0, 0) HadoopRDD.addLocalConfiguration("", 0, 0, 0, conf.value) - - val jCtxt = getJobContext() + + val jCtxt = getJobContext() getOutputCommitter().setupJob(jCtxt) } @@ -74,7 +74,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) val numfmt = NumberFormat.getInstance() numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) - + val outputName = "part-" + numfmt.format(splitID) val path = FileOutputFormat.getOutputPath(conf.value) val fs: FileSystem = { @@ -85,7 +85,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } } - getOutputCommitter().setupTask(getTaskContext()) + getOutputCommitter().setupTask(getTaskContext()) writer = getOutputFormat().getRecordWriter(fs, conf.value, outputName, Reporter.NULL) } @@ -103,18 +103,18 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def commit() { val taCtxt = getTaskContext() - val cmtr = getOutputCommitter() + val cmtr = getOutputCommitter() if (cmtr.needsTaskCommit(taCtxt)) { try { cmtr.commitTask(taCtxt) logInfo (taID + ": Committed") } catch { - case e: IOException => { + case e: IOException => { logError("Error committing the output of task: " + taID.value, e) cmtr.abortTask(taCtxt) throw e } - } + } } else { logWarning ("No need to commit output of task: " + taID.value) } @@ -144,7 +144,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } private def getJobContext(): JobContext = { - if (jobContext == null) { + if (jobContext == null) { jobContext = newJobContext(conf.value, jID.value) } jobContext @@ -175,7 +175,7 @@ object SparkHadoopWriter { val jobtrackerID = formatter.format(time) new JobID(jobtrackerID, id) } - + def createPathFromString(path: String, conf: JobConf): Path = { if (path == null) { throw new IllegalArgumentException("Output path is null") diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala index a2a871cbd3c31..5b14c4291d91a 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -44,12 +44,12 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg * configurable in the future. */ private var saslClient: SaslClient = Sasl.createSaslClient(Array[String](SparkSaslServer.DIGEST), - null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, + null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, new SparkSaslClientCallbackHandler(securityMgr)) /** * Used to initiate SASL handshake with server. - * @return response to challenge if needed + * @return response to challenge if needed */ def firstToken(): Array[Byte] = { synchronized { @@ -86,7 +86,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg } /** - * Disposes of any system resources or security-sensitive information the + * Disposes of any system resources or security-sensitive information the * SaslClient might be using. */ def dispose() { @@ -110,7 +110,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg private class SparkSaslClientCallbackHandler(securityMgr: SecurityManager) extends CallbackHandler { - private val userName: String = + private val userName: String = SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) private val secretKey = securityMgr.getSecretKey() private val userPassword: Array[Char] = @@ -138,7 +138,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg rc.setText(rc.getDefaultText()) } case cb: RealmChoiceCallback => {} - case cb: Callback => throw + case cb: Callback => throw new UnsupportedCallbackException(cb, "handle: Unrecognized SASL client callback") } } diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala index 11fcb2ae3a5c5..6161a6fb7ae85 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -64,7 +64,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi } /** - * Disposes of any system resources or security-sensitive information the + * Disposes of any system resources or security-sensitive information the * SaslServer might be using. */ def dispose() { @@ -88,7 +88,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi private class SparkSaslDigestCallbackHandler(securityMgr: SecurityManager) extends CallbackHandler { - private val userName: String = + private val userName: String = SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) override def handle(callbacks: Array[Callback]) { @@ -123,7 +123,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi ac.setAuthorizedID(authzid) } } - case cb: Callback => throw + case cb: Callback => throw new UnsupportedCallbackException(cb, "handle: Unrecognized SASL DIGEST-MD5 Callback") } } diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 4597595a838e3..f3f59e47c3e98 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -31,7 +31,7 @@ import com.google.common.io.Files * projects. * * TODO: See if we can move this to the test codebase by specifying - * test dependencies between projects. + * test dependencies between projects. */ private[spark] object TestUtils { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 2b32546c6854d..2659274c5e98e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -158,7 +158,7 @@ private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boo } def receiveBroadcast(): Boolean = { - // Receive meta-info about the size of broadcast data, + // Receive meta-info about the size of broadcast data, // the number of chunks it is divided into, etc. val metaId = BroadcastBlockId(id, "meta") var attemptId = 10 diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index c07838f798799..5da9615c9e9af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -43,7 +43,7 @@ private[spark] class ClientArguments(args: Array[String]) { // kill parameters var driverId: String = "" - + parse(args.toList) def parse(args: List[String]): Unit = args match { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index d35d5be73ff97..3836bf219ed3e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -32,8 +32,8 @@ private[spark] class WorkerArguments(args: Array[String]) { var memory = inferDefaultMemory() var masters: Array[String] = null var workDir: String = null - - // Check for settings in environment variables + + // Check for settings in environment variables if (System.getenv("SPARK_WORKER_PORT") != null) { port = System.getenv("SPARK_WORKER_PORT").toInt } @@ -49,7 +49,7 @@ private[spark] class WorkerArguments(args: Array[String]) { if (System.getenv("SPARK_WORKER_DIR") != null) { workDir = System.getenv("SPARK_WORKER_DIR") } - + parse(args.toList) def parse(args: List[String]): Unit = args match { @@ -78,7 +78,7 @@ private[spark] class WorkerArguments(args: Array[String]) { case ("--work-dir" | "-d") :: value :: tail => workDir = value parse(tail) - + case "--webui-port" :: IntParam(value) :: tail => webUiPort = value parse(tail) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 85200ab0e102d..49c1009cac2bf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -137,7 +137,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { .format(executor.appId, executor.execId)}>stdout stderr - + } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 16887d8892b31..6327ac01663f6 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -53,7 +53,7 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? - executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, + executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, false) case RegisterExecutorFailed(message) => diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index ceff3a067d72a..38be2c58b333f 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -34,7 +34,7 @@ object ExecutorExitCode { logging the exception. */ val UNCAUGHT_EXCEPTION_TWICE = 51 - /** The default uncaught exception handler was reached, and the uncaught exception was an + /** The default uncaught exception handler was reached, and the uncaught exception was an OutOfMemoryError. */ val OOM = 52 @@ -43,10 +43,10 @@ object ExecutorExitCode { /** TachyonStore failed to initialize after many attempts. */ val TACHYON_STORE_FAILED_TO_INITIALIZE = 54 - + /** TachyonStore failed to create a local temporary directory after many attempts. */ val TACHYON_STORE_FAILED_TO_CREATE_DIR = 55 - + def explainExitCode(exitCode: Int): String = { exitCode match { case UNCAUGHT_EXCEPTION => "Uncaught exception" @@ -57,7 +57,7 @@ object ExecutorExitCode { case TACHYON_STORE_FAILED_TO_INITIALIZE => "TachyonStore failed to initialize." case TACHYON_STORE_FAILED_TO_CREATE_DIR => "TachyonStore failed to create a local temporary directory." - case _ => + case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { " (died from signal " + (exitCode - 128) + "?)" diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala index 208e77073fd03..218ed7b5d2d39 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala @@ -38,7 +38,7 @@ private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: Class override def addURL(url: URL) { super.addURL(url) } - override def findClass(name: String): Class[_] = { + override def findClass(name: String): Class[_] = { super.findClass(name) } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 42c1200926fea..542dce65366b2 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -45,7 +45,7 @@ private[spark] class CsvSink(val property: Properties, val registry: MetricRegis 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 { diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 2f7576c53b482..3ffaaab23d0f5 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -248,14 +248,14 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } } - // outbox is used as a lock - ensure that it is always used as a leaf (since methods which + // outbox is used as a lock - ensure that it is always used as a leaf (since methods which // lock it are invoked in context of other locks) private val outbox = new Outbox() /* - This is orthogonal to whether we have pending bytes to write or not - and satisfies a slightly - different purpose. This flag is to see if we need to force reregister for write even when we + This is orthogonal to whether we have pending bytes to write or not - and satisfies a slightly + different purpose. This flag is to see if we need to force reregister for write even when we do not have any pending bytes to write to socket. - This can happen due to a race between adding pending buffers, and checking for existing of + This can happen due to a race between adding pending buffers, and checking for existing of data as detailed in https://github.com/mesos/spark/pull/791 */ private var needForceReregister = false diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionId.scala index ffaab677d411a..d579c165a1917 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionId.scala @@ -18,7 +18,7 @@ package org.apache.spark.network private[spark] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { - override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId + override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId } private[spark] object ConnectionId { @@ -26,9 +26,9 @@ private[spark] object ConnectionId { def createConnectionIdFromString(connectionIdString: String): ConnectionId = { val res = connectionIdString.split("_").map(_.trim()) if (res.size != 3) { - throw new Exception("Error converting ConnectionId string: " + connectionIdString + + throw new Exception("Error converting ConnectionId string: " + connectionIdString + " to a ConnectionId Object") } new ConnectionId(new ConnectionManagerId(res(0), res(1).toInt), res(2).toInt) - } + } } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index bdf586351ac14..cfee41c61362e 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -79,7 +79,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, private val serverChannel = ServerSocketChannel.open() // used to track the SendingConnections waiting to do SASL negotiation - private val connectionsAwaitingSasl = new HashMap[ConnectionId, SendingConnection] + private val connectionsAwaitingSasl = new HashMap[ConnectionId, SendingConnection] with SynchronizedMap[ConnectionId, SendingConnection] private val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] @@ -141,7 +141,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } finally { writeRunnableStarted.synchronized { writeRunnableStarted -= key - val needReregister = register || conn.resetForceReregister() + val needReregister = register || conn.resetForceReregister() if (needReregister && conn.changeInterestForWrite()) { conn.registerInterest() } @@ -509,7 +509,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, private def handleClientAuthentication( waitingConn: SendingConnection, - securityMsg: SecurityMessage, + securityMsg: SecurityMessage, connectionId : ConnectionId) { if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed for id: " + waitingConn.connectionId) @@ -530,7 +530,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } return } - var securityMsgResp = SecurityMessage.fromResponse(replyToken, + var securityMsgResp = SecurityMessage.fromResponse(replyToken, securityMsg.getConnectionId.toString()) var message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security message") @@ -546,7 +546,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } private def handleServerAuthentication( - connection: Connection, + connection: Connection, securityMsg: SecurityMessage, connectionId: ConnectionId) { if (!connection.isSaslComplete()) { @@ -561,7 +561,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } replyToken = connection.sparkSaslServer.response(securityMsg.getToken) if (connection.isSaslComplete()) { - logDebug("Server sasl completed: " + connection.connectionId) + logDebug("Server sasl completed: " + connection.connectionId) } else { logDebug("Server sasl not completed: " + connection.connectionId) } @@ -571,7 +571,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, var message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security Message") sendSecurityMessage(connection.getRemoteConnectionManagerId(), message) - } + } } catch { case e: Exception => { logError("Error in server auth negotiation: " + e) @@ -581,7 +581,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } } else { - logDebug("connection already established for this connection id: " + connection.connectionId) + logDebug("connection already established for this connection id: " + connection.connectionId) } } @@ -609,8 +609,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, return true } else { if (!conn.isSaslComplete()) { - // We could handle this better and tell the client we need to do authentication - // negotiation, but for now just ignore them. + // We could handle this better and tell the client we need to do authentication + // negotiation, but for now just ignore them. logError("message sent that is not security negotiation message on connection " + "not authenticated yet, ignoring it!!") return true @@ -709,11 +709,11 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } } else { - logDebug("Sasl already established ") + logDebug("Sasl already established ") } } - // allow us to add messages to the inbox for doing sasl negotiating + // allow us to add messages to the inbox for doing sasl negotiating private def sendSecurityMessage(connManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { val inetSocketAddress = new InetSocketAddress(connManagerId.host, connManagerId.port) @@ -772,7 +772,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, if (((clock.getTime() - startTime) >= (authTimeout * 1000)) && (!connection.isSaslComplete())) { // took to long to authenticate the connection, something probably went wrong - throw new Exception("Took to long for authentication to " + connectionManagerId + + throw new Exception("Took to long for authentication to " + connectionManagerId + ", waited " + authTimeout + "seconds, failing.") } } @@ -794,7 +794,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } case None => { - logError("no messageStatus for failed message id: " + message.id) + logError("no messageStatus for failed message id: " + message.id) } } } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index 9d9b9dbdd5331..4894ecd41f6eb 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -37,11 +37,11 @@ private[spark] object ConnectionManagerTest extends Logging{ "[size of msg in MB (integer)] [count] [await time in seconds)] ") System.exit(1) } - + if (args(0).startsWith("local")) { println("This runs only on a mesos cluster") } - + val sc = new SparkContext(args(0), "ConnectionManagerTest") val slavesFile = Source.fromFile(args(1)) val slaves = slavesFile.mkString.split("\n") @@ -50,7 +50,7 @@ private[spark] object ConnectionManagerTest extends Logging{ /* println("Slaves") */ /* slaves.foreach(println) */ val tasknum = if (args.length > 2) args(2).toInt else slaves.length - val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 + val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 val count = if (args.length > 4) args(4).toInt else 3 val awaitTime = (if (args.length > 5) args(5).toInt else 600 ).second println("Running " + count + " rounds of test: " + "parallel tasks = " + tasknum + ", " + @@ -64,16 +64,16 @@ private[spark] object ConnectionManagerTest extends Logging{ (0 until count).foreach(i => { val resultStrs = sc.parallelize(0 until tasknum, tasknum).map(i => { val connManager = SparkEnv.get.connectionManager - val thisConnManagerId = connManager.id - connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + val thisConnManagerId = connManager.id + connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { logInfo("Received [" + msg + "] from [" + id + "]") None }) val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip - - val startTime = System.currentTimeMillis + + val startTime = System.currentTimeMillis val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map{ slaveConnManagerId => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) @@ -84,7 +84,7 @@ private[spark] object ConnectionManagerTest extends Logging{ val results = futures.map(f => Await.result(f, awaitTime)) val finishTime = System.currentTimeMillis Thread.sleep(5000) - + val mb = size * results.size / 1024.0 / 1024.0 val ms = finishTime - startTime val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * @@ -92,11 +92,11 @@ private[spark] object ConnectionManagerTest extends Logging{ logInfo(resultStr) resultStr }).collect() - - println("---------------------") - println("Run " + i) + + println("---------------------") + println("Run " + i) resultStrs.foreach(println) - println("---------------------") + println("---------------------") }) } } diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 2b41c403b2e0a..9dc51e0d401f8 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.network import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} private[spark] object ReceiverTest { def main(args: Array[String]) { diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala index 0d9f743b3624b..a1dfc4094cca7 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala @@ -26,33 +26,33 @@ import org.apache.spark._ import org.apache.spark.network._ /** - * SecurityMessage is class that contains the connectionId and sasl token + * SecurityMessage is class that contains the connectionId and sasl token * used in SASL negotiation. SecurityMessage has routines for converting * it to and from a BufferMessage so that it can be sent by the ConnectionManager * and easily consumed by users when received. * The api was modeled after BlockMessage. * - * The connectionId is the connectionId of the client side. Since + * The connectionId is the connectionId of the client side. Since * message passing is asynchronous and its possible for the server side (receiving) - * to get multiple different types of messages on the same connection the connectionId - * is used to know which connnection the security message is intended for. - * + * to get multiple different types of messages on the same connection the connectionId + * is used to know which connnection the security message is intended for. + * * For instance, lets say we are node_0. We need to send data to node_1. The node_0 side * is acting as a client and connecting to node_1. SASL negotiation has to occur - * between node_0 and node_1 before node_1 trusts node_0 so node_0 sends a security message. - * node_1 receives the message from node_0 but before it can process it and send a response, - * some thread on node_1 decides it needs to send data to node_0 so it connects to node_0 - * and sends a security message of its own to authenticate as a client. Now node_0 gets - * the message and it needs to decide if this message is in response to it being a client - * (from the first send) or if its just node_1 trying to connect to it to send data. This + * between node_0 and node_1 before node_1 trusts node_0 so node_0 sends a security message. + * node_1 receives the message from node_0 but before it can process it and send a response, + * some thread on node_1 decides it needs to send data to node_0 so it connects to node_0 + * and sends a security message of its own to authenticate as a client. Now node_0 gets + * the message and it needs to decide if this message is in response to it being a client + * (from the first send) or if its just node_1 trying to connect to it to send data. This * is where the connectionId field is used. node_0 can lookup the connectionId to see if * it is in response to it being a client or if its in response to someone sending other data. - * + * * The format of a SecurityMessage as its sent is: * - Length of the ConnectionId - * - ConnectionId + * - ConnectionId * - Length of the token - * - Token + * - Token */ private[spark] class SecurityMessage() extends Logging { @@ -61,13 +61,13 @@ private[spark] class SecurityMessage() extends Logging { def set(byteArr: Array[Byte], newconnectionId: String) { if (byteArr == null) { - token = new Array[Byte](0) + token = new Array[Byte](0) } else { token = byteArr } connectionId = newconnectionId } - + /** * Read the given buffer and set the members of this class. */ @@ -91,17 +91,17 @@ private[spark] class SecurityMessage() extends Logging { buffer.clear() set(buffer) } - + def getConnectionId: String = { return connectionId } - + def getToken: Array[Byte] = { return token } - + /** - * Create a BufferMessage that can be sent by the ConnectionManager containing + * Create a BufferMessage that can be sent by the ConnectionManager containing * the security information from this class. * @return BufferMessage */ @@ -110,12 +110,12 @@ private[spark] class SecurityMessage() extends Logging { val buffers = new ArrayBuffer[ByteBuffer]() // 4 bytes for the length of the connectionId - // connectionId is of type char so multiple the length by 2 to get number of bytes + // connectionId is of type char so multiple the length by 2 to get number of bytes // 4 bytes for the length of token // token is a byte buffer so just take the length var buffer = ByteBuffer.allocate(4 + connectionId.length() * 2 + 4 + token.length) buffer.putInt(connectionId.length()) - connectionId.foreach((x: Char) => buffer.putChar(x)) + connectionId.foreach((x: Char) => buffer.putChar(x)) buffer.putInt(token.length) if (token.length > 0) { @@ -123,7 +123,7 @@ private[spark] class SecurityMessage() extends Logging { } buffer.flip() buffers += buffer - + var message = Message.createBufferMessage(buffers) logDebug("message total size is : " + message.size) message.isSecurityNeg = true @@ -136,7 +136,7 @@ private[spark] class SecurityMessage() extends Logging { } private[spark] object SecurityMessage { - + /** * Convert the given BufferMessage to a SecurityMessage by parsing the contents * of the BufferMessage and populating the SecurityMessage fields. diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index 4164e81d3a8ae..136c1912045aa 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -36,8 +36,8 @@ private[spark] class FileHeader ( if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) } else { - throw new Exception("too long header " + buf.readableBytes) - logInfo("too long header") + throw new Exception("too long header " + buf.readableBytes) + logInfo("too long header") } buf } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index eade07fbcbe37..cadd0c7ed19ba 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -44,7 +44,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { } } - /** + /** * Set a handler to be called when this PartialResult completes. Only one completion handler * is supported per PartialResult. */ @@ -60,7 +60,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { return this } - /** + /** * Set a handler to be called if this PartialResult's job fails. Only one failure handler * is supported per PartialResult. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 2306c9736b334..9ca971c8a4c27 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -52,7 +52,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Compute the standard deviation of this RDD's elements. */ def stdev(): Double = stats().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). */ @@ -123,13 +123,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * e.g. for the array * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] * e.g 1<=x<10 , 10<=x<20, 20<=x<50 - * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 - * + * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 + * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. - * buckets array must be at least two elements + * buckets array must be at least two elements * All NaN entries are treated the same. If you have a NaN bucket it must be * the maximum value of the last position and all NaN entries will be counted * in that bucket. diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index a84357b38414e..0c2cd7a24783b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -33,7 +33,7 @@ class PartitionerAwareUnionRDDPartition( val idx: Int ) extends Partition { var parents = rdds.map(_.partitions(idx)).toArray - + override val index = idx override def hashCode(): Int = idx diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 04c53d468465a..293cfb65643a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -54,7 +54,7 @@ private[scheduler] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] -case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent +case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] case class CompletionEvent( task: Task[_], diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 76f3e327d60b8..545fa453b7ccf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -1,107 +1,107 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler - -import java.util.concurrent.LinkedBlockingQueue - -import org.apache.spark.Logging - -/** - * Asynchronously passes SparkListenerEvents to registered SparkListeners. - * - * Until start() is called, all posted events are only buffered. Only after this listener bus - * has started will events be actually propagated to all attached listeners. This listener bus - * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). - */ -private[spark] class LiveListenerBus extends SparkListenerBus with Logging { - - /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ - private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) - private var queueFullErrorMessageLogged = false - private var started = false - private val listenerThread = new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return - } - postToAll(event) - } - } - } - - // Exposed for testing - @volatile private[spark] var stopCalled = false - - /** - * Start sending events to attached listeners. - * - * This first sends out all buffered events posted before this listener bus has started, then - * listens for any additional events asynchronously while the listener bus is still running. - * This should only be called once. - */ - def start() { - if (started) { - throw new IllegalStateException("Listener bus already started!") - } - listenerThread.start() - started = true - } - - def post(event: SparkListenerEvent) { - val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true - } - } - - /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time - * elapsed before the queue emptied. - */ - def waitUntilEmpty(timeoutMillis: Int): Boolean = { - val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { - if (System.currentTimeMillis > finishTime) { - return false - } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ - Thread.sleep(10) - } - true - } - - def stop() { - stopCalled = true - if (!started) { - throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") - } - post(SparkListenerShutdown) - listenerThread.join() - } -} +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.util.concurrent.LinkedBlockingQueue + +import org.apache.spark.Logging + +/** + * Asynchronously passes SparkListenerEvents to registered SparkListeners. + * + * Until start() is called, all posted events are only buffered. Only after this listener bus + * has started will events be actually propagated to all attached listeners. This listener bus + * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). + */ +private[spark] class LiveListenerBus extends SparkListenerBus with Logging { + + /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ + private val EVENT_QUEUE_CAPACITY = 10000 + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) + private var queueFullErrorMessageLogged = false + private var started = false + private val listenerThread = new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + postToAll(event) + } + } + } + + // Exposed for testing + @volatile private[spark] var stopCalled = false + + /** + * Start sending events to attached listeners. + * + * This first sends out all buffered events posted before this listener bus has started, then + * listens for any additional events asynchronously while the listener bus is still running. + * This should only be called once. + */ + def start() { + if (started) { + throw new IllegalStateException("Listener bus already started!") + } + listenerThread.start() + started = true + } + + def post(event: SparkListenerEvent) { + val eventAdded = eventQueue.offer(event) + if (!eventAdded && !queueFullErrorMessageLogged) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + + "rate at which tasks are being started by the scheduler.") + queueFullErrorMessageLogged = true + } + } + + /** + * Waits until there are no more events in the queue, or until the specified time has elapsed. + * Used for testing only. Returns true if the queue has emptied and false is the specified time + * elapsed before the queue emptied. + */ + def waitUntilEmpty(timeoutMillis: Int): Boolean = { + val finishTime = System.currentTimeMillis + timeoutMillis + while (!eventQueue.isEmpty) { + if (System.currentTimeMillis > finishTime) { + return false + } + /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify + * add overhead in the general case. */ + Thread.sleep(10) + } + true + } + + def stop() { + stopCalled = true + if (!started) { + throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") + } + post(SparkListenerShutdown) + listenerThread.join() + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 2fbbda5b76c74..ace9cd51c96b7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -240,7 +240,7 @@ object BlockFetcherIterator { override def numRemoteBlocks: Int = numRemote override def fetchWaitTime: Long = _fetchWaitTime override def remoteBytesRead: Long = _remoteBytesRead - + // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue // as they arrive. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a2a729130091f..df9bb4044e37a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -388,7 +388,7 @@ private[spark] class BlockManager( logDebug("Block " + blockId + " not found in memory") } } - + // Look for the block in Tachyon if (level.useOffHeap) { logDebug("Getting block " + blockId + " from tachyon") @@ -1031,7 +1031,7 @@ private[spark] class BlockManager( memoryStore.clear() diskStore.clear() if (tachyonInitialized) { - tachyonStore.clear() + tachyonStore.clear() } metadataCleaner.cancel() broadcastCleaner.cancel() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala index 7168ae18c2615..337b45b727dec 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala @@ -37,7 +37,7 @@ private[spark] class BlockMessage() { private var id: BlockId = null private var data: ByteBuffer = null private var level: StorageLevel = null - + def set(getBlock: GetBlock) { typ = BlockMessage.TYPE_GET_BLOCK id = getBlock.id @@ -75,13 +75,13 @@ private[spark] class BlockMessage() { idBuilder += buffer.getChar() } id = BlockId(idBuilder.toString) - + if (typ == BlockMessage.TYPE_PUT_BLOCK) { val booleanInt = buffer.getInt() val replication = buffer.getInt() level = StorageLevel(booleanInt, replication) - + val dataLength = buffer.getInt() data = ByteBuffer.allocate(dataLength) if (dataLength != buffer.remaining) { @@ -108,12 +108,12 @@ private[spark] class BlockMessage() { buffer.clear() set(buffer) } - + def getType: Int = typ def getId: BlockId = id def getData: ByteBuffer = data def getLevel: StorageLevel = level - + def toBufferMessage: BufferMessage = { val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() @@ -127,7 +127,7 @@ private[spark] class BlockMessage() { buffer = ByteBuffer.allocate(8).putInt(level.toInt).putInt(level.replication) buffer.flip() buffers += buffer - + buffer = ByteBuffer.allocate(4).putInt(data.remaining) buffer.flip() buffers += buffer @@ -140,7 +140,7 @@ private[spark] class BlockMessage() { buffers += data } - + /* println() println("BlockMessage: ") @@ -158,7 +158,7 @@ private[spark] class BlockMessage() { } override def toString: String = { - "BlockMessage [type = " + typ + ", id = " + id + ", level = " + level + + "BlockMessage [type = " + typ + ", id = " + id + ", level = " + level + ", data = " + (if (data != null) data.remaining.toString else "null") + "]" } } @@ -168,7 +168,7 @@ private[spark] object BlockMessage { val TYPE_GET_BLOCK: Int = 1 val TYPE_GOT_BLOCK: Int = 2 val TYPE_PUT_BLOCK: Int = 3 - + def fromBufferMessage(bufferMessage: BufferMessage): BlockMessage = { val newBlockMessage = new BlockMessage() newBlockMessage.set(bufferMessage) @@ -192,7 +192,7 @@ private[spark] object BlockMessage { newBlockMessage.set(gotBlock) newBlockMessage } - + def fromPutBlock(putBlock: PutBlock): BlockMessage = { val newBlockMessage = new BlockMessage() newBlockMessage.set(putBlock) @@ -206,7 +206,7 @@ private[spark] object BlockMessage { val bMsg = B.toBufferMessage val C = new BlockMessage() C.set(bMsg) - + println(B.getId + " " + B.getLevel) println(C.getId + " " + C.getLevel) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala index dc62b1efaa7d4..973d85c0a9b3a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala @@ -27,16 +27,16 @@ import org.apache.spark.network._ private[spark] class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { - + def this(bm: BlockMessage) = this(Array(bm)) def this() = this(null.asInstanceOf[Seq[BlockMessage]]) - def apply(i: Int) = blockMessages(i) + def apply(i: Int) = blockMessages(i) def iterator = blockMessages.iterator - def length = blockMessages.length + def length = blockMessages.length def set(bufferMessage: BufferMessage) { val startTime = System.currentTimeMillis @@ -62,15 +62,15 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) logDebug("Trying to convert buffer " + newBuffer + " to block message") val newBlockMessage = BlockMessage.fromByteBuffer(newBuffer) logDebug("Created " + newBlockMessage) - newBlockMessages += newBlockMessage + newBlockMessages += newBlockMessage buffer.position(buffer.position() + size) } val finishTime = System.currentTimeMillis logDebug("Converted block message array from buffer message in " + (finishTime - startTime) / 1000.0 + " s") - this.blockMessages = newBlockMessages + this.blockMessages = newBlockMessages } - + def toBufferMessage: BufferMessage = { val buffers = new ArrayBuffer[ByteBuffer]() @@ -83,7 +83,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) buffers ++= bufferMessage.buffers logDebug("Added " + bufferMessage) }) - + logDebug("Buffer list:") buffers.foreach((x: ByteBuffer) => logDebug("" + x)) /* @@ -103,13 +103,13 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) } private[spark] object BlockMessageArray { - + def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = { val newBlockMessageArray = new BlockMessageArray() newBlockMessageArray.set(bufferMessage) newBlockMessageArray } - + def main(args: Array[String]) { val blockMessages = (0 until 10).map { i => @@ -124,10 +124,10 @@ private[spark] object BlockMessageArray { } val blockMessageArray = new BlockMessageArray(blockMessages) println("Block message array created") - + val bufferMessage = blockMessageArray.toBufferMessage println("Converted to buffer message") - + val totalSize = bufferMessage.size val newBuffer = ByteBuffer.allocate(totalSize) newBuffer.clear() @@ -137,7 +137,7 @@ private[spark] object BlockMessageArray { buffer.rewind() }) newBuffer.flip - val newBufferMessage = Message.createBufferMessage(newBuffer) + val newBufferMessage = Message.createBufferMessage(newBuffer) println("Copied to new buffer message, size = " + newBufferMessage.size) val newBlockMessageArray = BlockMessageArray.fromBufferMessage(newBufferMessage) @@ -147,7 +147,7 @@ private[spark] object BlockMessageArray { case BlockMessage.TYPE_PUT_BLOCK => { val pB = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) println(pB) - } + } case BlockMessage.TYPE_GET_BLOCK => { val gB = new GetBlock(blockMessage.getId) println(gB) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index e1a1f209c9282..9ce0398d010a8 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -136,7 +136,7 @@ private[spark] object JettyUtils extends Logging { private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) filters.foreach { - case filter : String => + case filter : String => if (!filter.isEmpty) { logInfo("Adding filter: " + filter) val holder : FilterHolder = new FilterHolder() @@ -151,7 +151,7 @@ private[spark] object JettyUtils extends Logging { if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) } } - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a487924effbff..a7cf04b3cbb86 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -48,7 +48,7 @@ private[spark] object UIUtils { case _ =>
  • Storage
  • } val environment = page match { - case Environment => + case Environment =>
  • Environment
  • case _ =>
  • Environment
  • } diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index cdbbc65292188..2d05e09b10948 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -45,7 +45,7 @@ private[spark] object ClosureCleaner extends Logging { private def isClosure(cls: Class[_]): Boolean = { cls.getName.contains("$anonfun$") } - + // Get a list of the classes of the outer objects of a given closure object, obj; // the outer objects are defined as any closures that obj is nested within, plus // possibly the class that the outermost closure is in, if any. We stop searching @@ -63,7 +63,7 @@ private[spark] object ClosureCleaner extends Logging { } Nil } - + // Get a list of the outer objects for a given closure object. private def getOuterObjects(obj: AnyRef): List[AnyRef] = { for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { @@ -76,7 +76,7 @@ private[spark] object ClosureCleaner extends Logging { } Nil } - + private def getInnerClasses(obj: AnyRef): List[Class[_]] = { val seen = Set[Class[_]](obj.getClass) var stack = List[Class[_]](obj.getClass) @@ -92,7 +92,7 @@ private[spark] object ClosureCleaner extends Logging { } return (seen - obj.getClass).toList } - + private def createNullValue(cls: Class[_]): AnyRef = { if (cls.isPrimitive) { new java.lang.Byte(0: Byte) // Should be convertible to any primitive type @@ -100,13 +100,13 @@ private[spark] object ClosureCleaner extends Logging { null } } - + def clean(func: AnyRef) { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) val outerObjects = getOuterObjects(func) - + val accessedFields = Map[Class[_], Set[String]]() for (cls <- outerClasses) accessedFields(cls) = Set[String]() @@ -143,7 +143,7 @@ private[spark] object ClosureCleaner extends Logging { field.set(outer, value) } } - + if (outer != null) { // logInfo("2: Setting $outer on " + func.getClass + " to " + outer); val field = func.getClass.getDeclaredField("$outer") @@ -151,7 +151,7 @@ private[spark] object ClosureCleaner extends Logging { field.set(func, outer) } } - + private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { // logInfo("Creating a " + cls + " with outer = " + outer) if (!inInterpreter) { @@ -192,7 +192,7 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor } } } - + override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { // Check for calls a getter method for a variable in an interpreter wrapper object. @@ -209,12 +209,12 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { var myName: String = null - + override def visit(version: Int, access: Int, name: String, sig: String, superName: String, interfaces: Array[String]) { myName = name } - + override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { new MethodVisitor(ASM4) { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index d990fd49ef834..f2396f7c80a35 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -611,7 +611,7 @@ private[spark] object JsonProtocol { val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize - rddInfo.tachyonSize = tachyonSize + rddInfo.tachyonSize = tachyonSize rddInfo.diskSize = diskSize rddInfo } diff --git a/core/src/main/scala/org/apache/spark/util/NextIterator.scala b/core/src/main/scala/org/apache/spark/util/NextIterator.scala index 8266e5e495efc..e5c732a5a559b 100644 --- a/core/src/main/scala/org/apache/spark/util/NextIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/NextIterator.scala @@ -19,7 +19,7 @@ package org.apache.spark.util /** Provides a basic/boilerplate Iterator implementation. */ private[spark] abstract class NextIterator[U] extends Iterator[U] { - + private var gotNext = false private var nextValue: U = _ private var closed = false @@ -34,7 +34,7 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { * This convention is required because `null` may be a valid value, * and using `Option` seems like it might create unnecessary Some/None * instances, given some iterators might be called in a tight loop. - * + * * @return U, or set 'finished' when done */ protected def getNext(): U diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala index 732748a7ff82b..d80eed455c427 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -62,10 +62,10 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { if (n == 0) { mu = other.mu m2 = other.m2 - n = other.n + n = other.n maxValue = other.maxValue minValue = other.minValue - } else if (other.n != 0) { + } else if (other.n != 0) { val delta = other.mu - mu if (other.n * 10 < n) { mu = mu + (delta * other.n) / (n + other.n) diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index 3c8f94a416c65..1a647fa1c9d84 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -136,7 +136,7 @@ object Vector { def ones(length: Int) = Vector(length, _ => 1) /** - * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers + * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. */ def random(length: Int, random: Random = new XORShiftRandom()) = diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 8a4cdea2fa7b1..7f220383f9f8b 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -25,28 +25,28 @@ import scala.util.hashing.MurmurHash3 import org.apache.spark.util.Utils.timeIt /** - * This class implements a XORShift random number generator algorithm + * This class implements a XORShift random number generator algorithm * Source: * Marsaglia, G. (2003). Xorshift RNGs. Journal of Statistical Software, Vol. 8, Issue 14. * @see Paper * This implementation is approximately 3.5 times faster than * {@link java.util.Random java.util.Random}, partly because of the algorithm, but also due - * to renouncing thread safety. JDK's implementation uses an AtomicLong seed, this class + * to renouncing thread safety. JDK's implementation uses an AtomicLong seed, this class * uses a regular Long. We can forgo thread safety since we use a new instance of the RNG * for each thread. */ private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) { - + def this() = this(System.nanoTime) private var seed = XORShiftRandom.hashSeed(init) // we need to just override next - this will be called by nextInt, nextDouble, // nextGaussian, nextLong, etc. - override protected def next(bits: Int): Int = { + override protected def next(bits: Int): Int = { var nextSeed = seed ^ (seed << 21) nextSeed ^= (nextSeed >>> 35) - nextSeed ^= (nextSeed << 4) + nextSeed ^= (nextSeed << 4) seed = nextSeed (nextSeed & ((1L << bits) -1)).asInstanceOf[Int] } @@ -89,7 +89,7 @@ private[spark] object XORShiftRandom { val million = 1e6.toInt val javaRand = new JavaRandom(seed) val xorRand = new XORShiftRandom(seed) - + // this is just to warm up the JIT - we're not timing anything timeIt(1e6.toInt) { javaRand.nextInt() @@ -97,9 +97,9 @@ private[spark] object XORShiftRandom { } val iters = timeIt(numIters)(_) - + /* Return results as a map instead of just printing to screen - in case the user wants to do something with them */ + in case the user wants to do something with them */ Map("javaTime" -> iters {javaRand.nextInt()}, "xorTime" -> iters {xorRand.nextInt()}) diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala index c5f24c66ce0c1..c645e4cbe8132 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -37,7 +37,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -54,14 +54,14 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { assert(securityManagerBad.isAuthenticationEnabled() === true) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) - intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) } actorSystem.shutdown() @@ -75,7 +75,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -91,7 +91,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { badconf.set("spark.authenticate.secret", "good") val securityManagerBad = new SecurityManager(badconf); - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( @@ -127,7 +127,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -180,7 +180,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -204,8 +204,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) - intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) } actorSystem.shutdown() diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 9cbdfc54a3dc8..7f59bdcce4cc7 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -39,7 +39,7 @@ class DriverSuite extends FunSuite with Timeouts { failAfter(60 seconds) { Utils.executeAndGetOutput( Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), - new File(sparkHome), + new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } } diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index aee9ab9091dac..d651fbbac4e97 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -45,7 +45,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val pw = new PrintWriter(textFile) pw.println("100") pw.close() - + val jarFile = new File(tmpDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) @@ -53,7 +53,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val jarEntry = new JarEntry(textFile.getName) jar.putNextEntry(jarEntry) - + val in = new FileInputStream(textFile) val buffer = new Array[Byte](10240) var nRead = 0 diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 01af94077144a..b9b668d3cc62a 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -106,7 +106,7 @@ class FileSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath - val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) + val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) nums.saveAsSequenceFile(outputDir) // Try reading the output back as a SequenceFile val output = sc.sequenceFile[IntWritable, Text](outputDir) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala index 0b5ed6d77034b..5e538d6fab2a1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala @@ -45,4 +45,4 @@ class WorkerWatcherSuite extends FunSuite { actorRef.underlyingActor.receive(new DisassociatedEvent(null, otherAkkaAddress, false)) assert(!actorRef.underlyingActor.isShutDown) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 09e35bfc8f85f..e89b296d41026 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -42,7 +42,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { override def beforeAll() { sc = new SparkContext("local", "test") - + // Set the block size of local file system to test whether files are split right or not. sc.hadoopConfiguration.setLong("fs.local.block.size", 32) } diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index a4381a8b974df..4df36558b6d4b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -34,14 +34,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === "2") assert(slices(2).mkString(",") === "3") } - + test("one slice") { val data = Array(1, 2, 3) val slices = ParallelCollectionRDD.slice(data, 1) assert(slices.size === 1) assert(slices(0).mkString(",") === "1,2,3") } - + test("equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9) val slices = ParallelCollectionRDD.slice(data, 3) @@ -50,7 +50,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === "4,5,6") assert(slices(2).mkString(",") === "7,8,9") } - + test("non-equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) val slices = ParallelCollectionRDD.slice(data, 3) @@ -77,14 +77,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === (33 to 66).mkString(",")) assert(slices(2).mkString(",") === (67 to 100).mkString(",")) } - + test("empty data") { val data = new Array[Int](0) val slices = ParallelCollectionRDD.slice(data, 5) assert(slices.size === 5) for (slice <- slices) assert(slice.size === 0) } - + test("zero slices") { val data = Array(1, 2, 3) intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, 0) } @@ -94,7 +94,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = Array(1, 2, 3) intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, -5) } } - + test("exclusive ranges sliced into ranges") { val data = 1 until 100 val slices = ParallelCollectionRDD.slice(data, 3) @@ -102,7 +102,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[Range])) } - + test("inclusive ranges sliced into ranges") { val data = 1 to 100 val slices = ParallelCollectionRDD.slice(data, 3) @@ -124,7 +124,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(range.step === 1, "slice " + i + " step") } } - + test("random array tests") { val gen = for { d <- arbitrary[List[Int]] @@ -141,7 +141,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } check(prop) } - + test("random exclusive range tests") { val gen = for { a <- Gen.choose(-100, 100) @@ -177,7 +177,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } check(prop) } - + test("exclusive ranges of longs") { val data = 1L until 100L val slices = ParallelCollectionRDD.slice(data, 3) @@ -185,7 +185,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("inclusive ranges of longs") { val data = 1L to 100L val slices = ParallelCollectionRDD.slice(data, 3) @@ -193,7 +193,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("exclusive ranges of doubles") { val data = 1.0 until 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) @@ -201,7 +201,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("inclusive ranges of doubles") { val data = 1.0 to 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index dc704e07a81de..4cdccdda6f72e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -216,7 +216,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc test("onTaskGettingResult() called when result fetched remotely") { val listener = new SaveTaskEvents sc.addSparkListener(listener) - + // Make a task whose result is larger than the akka frame size System.setProperty("spark.akka.frameSize", "1") val akkaFrameSize = @@ -236,7 +236,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc test("onTaskGettingResult() not called when result sent directly") { val listener = new SaveTaskEvents sc.addSparkListener(listener) - + // Make a task whose result is larger than the akka frame size val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 356e28dd19bc5..2fb750d9ee378 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -264,7 +264,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin test("Scheduler does not always schedule tasks on the same workers") { sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) + val taskScheduler = new TaskSchedulerImpl(sc) taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. val dagScheduler = new DAGScheduler(sc, taskScheduler) { diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 45c322427930d..2f9739f940dc6 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -33,8 +33,8 @@ class UISuite extends FunSuite { val server = new Server(startPort) Try { server.start() } match { - case Success(s) => - case Failure(e) => + case Success(s) => + case Failure(e) => // Either case server port is busy hence setup for test complete } val serverInfo1 = JettyUtils.startJettyServer( diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 439e5644e20a3..d7e48e633e0ee 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -69,7 +69,7 @@ object TestObject { class TestClass extends Serializable { var x = 5 - + def getX = x def run(): Int = { diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index e1446cbc90bdb..32d74d0500b72 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -32,7 +32,7 @@ class NextIteratorSuite extends FunSuite with ShouldMatchers { i.hasNext should be === false intercept[NoSuchElementException] { i.next() } } - + test("two iterations") { val i = new StubIterator(Buffer(1, 2)) i.hasNext should be === true @@ -70,7 +70,7 @@ class NextIteratorSuite extends FunSuite with ShouldMatchers { class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { var closeCalled = 0 - + override def getNext() = { if (ints.size == 0) { finished = true diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 757476efdb789..39199a1a17ccd 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -29,12 +29,12 @@ class XORShiftRandomSuite extends FunSuite with ShouldMatchers { val xorRand = new XORShiftRandom(seed) val hundMil = 1e8.toInt } - + /* - * This test is based on a chi-squared test for randomness. The values are hard-coded + * This test is based on a chi-squared test for randomness. The values are hard-coded * so as not to create Spark's dependency on apache.commons.math3 just to call one * method for calculating the exact p-value for a given number of random numbers - * and bins. In case one would want to move to a full-fledged test based on + * and bins. In case one would want to move to a full-fledged test based on * apache.commons.math3, the relevant class is here: * org.apache.commons.math3.stat.inference.ChiSquareTest */ @@ -49,19 +49,19 @@ class XORShiftRandomSuite extends FunSuite with ShouldMatchers { // populate bins based on modulus of the random number times(f.hundMil) {bins(math.abs(f.xorRand.nextInt) % 10) += 1} - /* since the seed is deterministic, until the algorithm is changed, we know the result will be - * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, - * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) - * significance level. However, should the RNG implementation change, the test should still - * pass at the same significance level. The chi-squared test done in R gave the following + /* since the seed is deterministic, until the algorithm is changed, we know the result will be + * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, + * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) + * significance level. However, should the RNG implementation change, the test should still + * pass at the same significance level. The chi-squared test done in R gave the following * results: * > chisq.test(c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, * 10000790, 10002286, 9998699)) * Chi-squared test for given probabilities - * data: c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, + * data: c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, * 10002286, 9998699) * X-squared = 11.975, df = 9, p-value = 0.2147 - * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million + * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million * random numbers * and 10 bins will happen at X-squared of ~16.9196. So, the test will fail if X-squared * is greater than or equal to that number. diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 41e813d48c7b8..1204cfba39f77 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -48,41 +48,41 @@ import org.apache.spark.streaming.dstream._ * @param storageLevel RDD storage level. */ -private[streaming] +private[streaming] class MQTTInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, brokerUrl: String, topic: String, storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_) with Logging { - + def getReceiver(): NetworkReceiver[T] = { new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]] } } -private[streaming] +private[streaming] class MQTTReceiver(brokerUrl: String, topic: String, storageLevel: StorageLevel ) extends NetworkReceiver[Any] { lazy protected val blockGenerator = new BlockGenerator(storageLevel) - + def onStop() { blockGenerator.stop() } - + def onStart() { blockGenerator.start() - // Set up persistence for messages + // Set up persistence for messages var peristance: MqttClientPersistence = new MemoryPersistence() // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance var client: MqttClient = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance) - // Connect to MqttBroker + // Connect to MqttBroker client.connect() // Subscribe to Mqtt topic @@ -91,7 +91,7 @@ class MQTTReceiver(brokerUrl: String, // Callback automatically triggers as and when new message arrives on specified topic var callback: MqttCallback = new MqttCallback() { - // Handles Mqtt message + // Handles Mqtt message override def messageArrived(arg0: String, arg1: MqttMessage) { blockGenerator += new String(arg1.getPayload()) } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 3316b6dc39d6b..843a4a7a9ad72 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel * @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. -* +* * If no Authorization object is provided, initializes OAuth authorization using the system * properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. */ @@ -42,13 +42,13 @@ class TwitterInputDStream( filters: Seq[String], storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - + private def createOAuthAuthorization(): Authorization = { new OAuthAuthorization(new ConfigurationBuilder().build()) } private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) - + override def getReceiver(): NetworkReceiver[Status] = { new TwitterReceiver(authorization, filters, storageLevel) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 377d9d6bd5e72..5635287694ee2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -172,7 +172,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali "EdgeDirection.Either instead.") } } - + /** * Join the vertices with an RDD and then apply a function from the * the vertex and RDD entry to a new vertex value. The input table diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index 6386306c048fc..a467ca1ae715a 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -55,7 +55,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { } } } - + test ("filter") { withSpark { sc => val n = 5 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index e41d9bbe18c37..7f6d94571b5ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -30,7 +30,7 @@ import org.apache.spark.mllib.linalg.Vector trait Optimizer extends Serializable { /** - * Solve the provided convex optimization problem. + * Solve the provided convex optimization problem. */ def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 3bd0017aa196a..d969e7aa60061 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** - * GeneralizedLinearModel (GLM) represents a model trained using + * GeneralizedLinearModel (GLM) represents a model trained using * GeneralizedLinearAlgorithm. GLMs consist of a weight vector and * an intercept. * @@ -38,7 +38,7 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double /** * Predict the result given a data point and the weights learned. - * + * * @param dataMatrix Row vector containing the features for this data point * @param weightMatrix Column vector containing the weights of the model * @param intercept Intercept of the model. diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index a30dcfdcecf27..687e85ca94d3c 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -35,7 +35,7 @@ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ * A ClassLoader that reads classes from a Hadoop FileSystem or HTTP URI, * used to load classes defined by the interpreter when the REPL is used. * Allows the user to specify if user class path should be first - */ + */ class ExecutorClassLoader(classUri: String, parent: ClassLoader, userClassPathFirst: Boolean) extends ClassLoader { val uri = new URI(classUri) @@ -94,7 +94,7 @@ class ExecutorClassLoader(classUri: String, parent: ClassLoader, case e: Exception => None } } - + def readAndTransformClass(name: String, in: InputStream): Array[Byte] = { if (name.startsWith("line") && name.endsWith("$iw$")) { // Class seems to be an interpreter "wrapper" object storing a val or var. diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala index 8f61a5e835044..419796b68b113 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -187,7 +187,7 @@ trait SparkImports { if (currentImps contains imv) addWrapper() val objName = req.lineRep.readPath val valName = "$VAL" + newValId(); - + if(!code.toString.endsWith(".`" + imv + "`;\n")) { // Which means already imported code.append("val " + valName + " = " + objName + ".INSTANCE;\n") code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 17118499d0c87..1f3fab09e9566 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -28,7 +28,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { override def toString = s"CAST($child, $dataType)" type EvaluatedType = Any - + def nullOrCast[T](a: Any, func: T => Any): Any = if(a == null) { null } else { @@ -40,7 +40,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case BinaryType => nullOrCast[Array[Byte]](_, new String(_, "UTF-8")) case _ => nullOrCast[Any](_, _.toString) } - + // BinaryConverter def castToBinary: Any => Any = child.dataType match { case StringType => nullOrCast[String](_, _.getBytes("UTF-8")) @@ -58,7 +58,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case DoubleType => nullOrCast[Double](_, _ != 0) case FloatType => nullOrCast[Float](_, _ != 0) } - + // TimestampConverter def castToTimestamp: Any => Any = child.dataType match { case StringType => nullOrCast[String](_, s => { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 8a1db8e796816..dd9332ada80dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -86,7 +86,7 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Numeric children expressions. Those expressions are supposed + * Evaluation helper function for 2 Numeric children expressions. Those expressions are supposed * to be in the same data type, and also the return type. * Either one of the expressions result is null, the evaluation result should be null. */ @@ -120,7 +120,7 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Fractional children expressions. Those expressions are + * Evaluation helper function for 2 Fractional children expressions. Those expressions are * supposed to be in the same data type, and also the return type. * Either one of the expressions result is null, the evaluation result should be null. */ @@ -153,7 +153,7 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Integral children expressions. Those expressions are + * Evaluation helper function for 2 Integral children expressions. Those expressions are * supposed to be in the same data type, and also the return type. * Either one of the expressions result is null, the evaluation result should be null. */ @@ -186,12 +186,12 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Comparable children expressions. Those expressions are + * Evaluation helper function for 2 Comparable children expressions. Those expressions are * supposed to be in the same data type, and the return type should be Integer: * Negative value: 1st argument less than 2nd argument * Zero: 1st argument equals 2nd argument * Positive value: 1st argument greater than 2nd argument - * + * * Either one of the expressions result is null, the evaluation result should be null. */ @inline @@ -213,7 +213,7 @@ abstract class Expression extends TreeNode[Expression] { null } else { e1.dataType match { - case i: NativeType => + case i: NativeType => f.asInstanceOf[(Ordering[i.JvmType], i.JvmType, i.JvmType) => Boolean]( i.ordering, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) case other => sys.error(s"Type $other does not support ordered operations") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index a27c71db1b999..ddc16ce87b895 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -28,19 +28,19 @@ trait StringRegexExpression { self: BinaryExpression => type EvaluatedType = Any - + def escape(v: String): String def matches(regex: Pattern, str: String): Boolean - + def nullable: Boolean = true def dataType: DataType = BooleanType - - // try cache the pattern for Literal + + // try cache the pattern for Literal private lazy val cache: Pattern = right match { case x @ Literal(value: String, StringType) => compile(value) case _ => null } - + protected def compile(str: String): Pattern = if(str == null) { null } else { @@ -49,7 +49,7 @@ trait StringRegexExpression { } protected def pattern(str: String) = if(cache == null) compile(str) else cache - + override def eval(input: Row): Any = { val l = left.eval(input) if (l == null) { @@ -73,11 +73,11 @@ trait StringRegexExpression { /** * Simple RegEx pattern matching function */ -case class Like(left: Expression, right: Expression) +case class Like(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - + def symbol = "LIKE" - + // replace the _ with .{1} exactly match 1 time of any character // replace the % with .*, match 0 or more times with any character override def escape(v: String) = { @@ -98,19 +98,19 @@ case class Like(left: Expression, right: Expression) sb.append(Pattern.quote(Character.toString(n))); } } - + i += 1 } - + sb.toString() } - + override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() } -case class RLike(left: Expression, right: Expression) +case class RLike(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - + def symbol = "RLIKE" override def escape(v: String): String = v override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index cdeb01a9656f4..da34bd3a21503 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -55,9 +55,9 @@ case object BooleanType extends NativeType { case object TimestampType extends NativeType { type JvmType = Timestamp - + @transient lazy val tag = typeTag[JvmType] - + val ordering = new Ordering[JvmType] { def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 888a19d79f7e4..2cd0d2b0e1385 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -144,7 +144,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like "b%", false) checkEvaluation("abc" like "bc%", false) } - + test("LIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) checkEvaluation("abcd" like regEx, null, new GenericRow(Array[Any](null))) @@ -164,7 +164,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("RLIKE literal Regular Expression") { checkEvaluation("abdef" rlike "abdef", true) checkEvaluation("abbbbc" rlike "a.*c", true) - + checkEvaluation("fofo" rlike "^fo", true) checkEvaluation("fo\no" rlike "^fo\no$", true) checkEvaluation("Bn" rlike "^Ba*n", true) @@ -196,9 +196,9 @@ class ExpressionEvaluationSuite extends FunSuite { evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) } } - + test("data type casting") { - + val sts = "1970-01-01 00:00:01.0" val ts = Timestamp.valueOf(sts) @@ -236,7 +236,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("23" cast ShortType, 23) checkEvaluation("2012-12-11" cast DoubleType, null) checkEvaluation(Literal(123) cast IntegerType, 123) - + intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 65eae3357a21e..1cbf973c34917 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -56,4 +56,4 @@ class ScalaReflectionRelationSuite extends FunSuite { val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 93023e8dced57..ac56ff709c1c4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -59,7 +59,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) } } -private[streaming] +private[streaming] object Checkpoint extends Logging { val PREFIX = "checkpoint-" val REGEX = (PREFIX + """([\d]+)([\w\.]*)""").r @@ -79,7 +79,7 @@ object Checkpoint extends Logging { def sortFunc(path1: Path, path2: Path): Boolean = { val (time1, bk1) = path1.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } val (time2, bk2) = path2.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } - (time1 < time2) || (time1 == time2 && bk1) + (time1 < time2) || (time1 == time2 && bk1) } val path = new Path(checkpointDir) @@ -95,7 +95,7 @@ object Checkpoint extends Logging { } } else { logInfo("Checkpoint directory " + path + " does not exist") - Seq.empty + Seq.empty } } } @@ -160,7 +160,7 @@ class CheckpointWriter( }) } - // All done, print success + // All done, print success val finishTime = System.currentTimeMillis() logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + checkpointFile + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") @@ -227,14 +227,14 @@ object CheckpointReader extends Logging { { val checkpointPath = new Path(checkpointDir) def fs = checkpointPath.getFileSystem(hadoopConf) - - // Try to find the checkpoint files + + // Try to find the checkpoint files val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, fs).reverse if (checkpointFiles.isEmpty) { return None } - // Try to read the checkpoint files in the order + // Try to read the checkpoint files in the order logInfo("Checkpoint files found: " + checkpointFiles.mkString(",")) val compressionCodec = CompressionCodec.createCodec(conf) checkpointFiles.foreach(file => { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala index 16479a01272aa..ad4f3fdd14ad6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala @@ -20,11 +20,11 @@ package org.apache.spark.streaming private[streaming] class Interval(val beginTime: Time, val endTime: Time) { def this(beginMs: Long, endMs: Long) = this(new Time(beginMs), new Time(endMs)) - + def duration(): Duration = endTime - beginTime def + (time: Duration): Interval = { - new Interval(beginTime + time, endTime + time) + new Interval(beginTime + time, endTime + time) } def - (time: Duration): Interval = { @@ -40,9 +40,9 @@ class Interval(val beginTime: Time, val endTime: Time) { } def <= (that: Interval) = (this < that || this == that) - + def > (that: Interval) = !(this <= that) - + def >= (that: Interval) = !(this < that) override def toString = "[" + beginTime + ", " + endTime + "]" diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index 2678334f53844..6a6b00a778b48 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -32,7 +32,7 @@ case class Time(private val millis: Long) { def <= (that: Time): Boolean = (this.millis <= that.millis) def > (that: Time): Boolean = (this.millis > that.millis) - + def >= (that: Time): Boolean = (this.millis >= that.millis) def + (that: Duration): Time = new Time(millis + that.milliseconds) @@ -43,7 +43,7 @@ case class Time(private val millis: Long) { def floor(that: Duration): Time = { val t = that.milliseconds - val m = math.floor(this.millis / t).toLong + val m = math.floor(this.millis / t).toLong new Time(m * t) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index 903e3f3c9b713..f33c0ceafdf42 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -51,7 +51,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) .map(x => (x._1, x._2.getCheckpointFile.get)) logDebug("Current checkpoint files:\n" + checkpointFiles.toSeq.mkString("\n")) - // Add the checkpoint files to the data to be serialized + // Add the checkpoint files to the data to be serialized if (!checkpointFiles.isEmpty) { currentCheckpointFiles.clear() currentCheckpointFiles ++= checkpointFiles diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 8a6051622e2d5..e878285f6a854 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -232,7 +232,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas } logDebug("Accepted " + path) } catch { - case fnfe: java.io.FileNotFoundException => + case fnfe: java.io.FileNotFoundException => logWarning("Error finding new files", fnfe) reset() return false diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala index 97325f8ea3117..6376cff78b78a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala @@ -31,11 +31,11 @@ class QueueInputDStream[T: ClassTag]( oneAtATime: Boolean, defaultRDD: RDD[T] ) extends InputDStream[T](ssc) { - + override def start() { } - + override def stop() { } - + override def compute(validTime: Time): Option[RDD[T]] = { val buffer = new ArrayBuffer[RDD[T]]() if (oneAtATime && queue.size > 0) { @@ -55,5 +55,5 @@ class QueueInputDStream[T: ClassTag]( None } } - + } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index 44eb2750c6c7a..f5984d03c5342 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -47,7 +47,7 @@ object ReceiverSupervisorStrategy { * the API for pushing received data into Spark Streaming for being processed. * * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * + * * @example {{{ * class MyActor extends Actor with Receiver{ * def receive { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index c5ef2cc8c390d..39145a3ab081a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -19,34 +19,34 @@ package org.apache.spark.streaming.util private[streaming] trait Clock { - def currentTime(): Long + def currentTime(): Long def waitTillTime(targetTime: Long): Long } private[streaming] class SystemClock() extends Clock { - + val minPollTime = 25L - + def currentTime(): Long = { System.currentTimeMillis() - } - + } + def waitTillTime(targetTime: Long): Long = { var currentTime = 0L currentTime = System.currentTimeMillis() - + var waitTime = targetTime - currentTime if (waitTime <= 0) { return currentTime } - + val pollTime = { if (waitTime / 10.0 > minPollTime) { (waitTime / 10.0).toLong } else { - minPollTime - } + minPollTime + } } while (true) { @@ -55,7 +55,7 @@ class SystemClock() extends Clock { if (waitTime <= 0) { return currentTime } - val sleepTime = + val sleepTime = if (waitTime < pollTime) { waitTime } else { @@ -69,7 +69,7 @@ class SystemClock() extends Clock { private[streaming] class ManualClock() extends Clock { - + var time = 0L def currentTime() = time @@ -85,13 +85,13 @@ class ManualClock() extends Clock { this.synchronized { time += timeToAdd this.notifyAll() - } + } } def waitTillTime(targetTime: Long): Long = { this.synchronized { while (time < targetTime) { this.wait(100) - } + } } currentTime() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index 07021ebb5802a..bd1df55cf70f5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -25,8 +25,8 @@ import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { - /** - * Splits lines and counts the words in them using specialized object-to-long hashmap + /** + * Splits lines and counts the words in them using specialized object-to-long hashmap * (to avoid boxing-unboxing overhead of Long in java/scala HashMap) */ def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = { @@ -55,13 +55,13 @@ object RawTextHelper { map.toIterator.map{case (k, v) => (k, v)} } - /** + /** * Gets the top k words in terms of word counts. Assumes that each word exists only once * in the `data` iterator (that is, the counts have been reduced). */ def topK(data: Iterator[(String, Long)], k: Int): Iterator[(String, Long)] = { val taken = new Array[(String, Long)](k) - + var i = 0 var len = 0 var done = false @@ -93,7 +93,7 @@ object RawTextHelper { } taken.toIterator } - + /** * Warms up the SparkContext in master and slave by running tasks to force JIT kick in * before real workload starts. @@ -106,11 +106,11 @@ object RawTextHelper { .count() } } - - def add(v1: Long, v2: Long) = (v1 + v2) - def subtract(v1: Long, v2: Long) = (v1 - v2) + def add(v1: Long, v2: Long) = (v1 + v2) + + def subtract(v1: Long, v2: Long) = (v1 - v2) - def max(v1: Long, v2: Long) = math.max(v1, v2) + def max(v1: Long, v2: Long) = math.max(v1, v2) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index f71938ac55ccb..e016377c94c0d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -22,10 +22,10 @@ import org.apache.spark.Logging private[streaming] class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: String) extends Logging { - + private val thread = new Thread("RecurringTimer - " + name) { setDaemon(true) - override def run() { loop } + override def run() { loop } } @volatile private var prevTime = -1L @@ -104,11 +104,11 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: private[streaming] object RecurringTimer { - + def main(args: Array[String]) { var lastRecurTime = 0L val period = 1000 - + def onRecur(time: Long) { val currentTime = System.currentTimeMillis() println("" + currentTime + ": " + (currentTime - lastRecurTime)) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 13fa64894b773..a0b1bbc34fa7c 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1673,7 +1673,7 @@ public void testSocketTextStream() { @Test public void testSocketString() { - + class Converter implements Function> { public Iterable call(InputStream in) throws IOException { BufferedReader reader = new BufferedReader(new InputStreamReader(in)); From f99401a6308d5b9a9259d7597a35ba92f927aa50 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 10 Apr 2014 16:20:33 -0700 Subject: [PATCH 08/61] [SQL] Improve column pruning in the optimizer. Author: Michael Armbrust Closes #378 from marmbrus/columnPruning and squashes the following commits: 779da56 [Michael Armbrust] More consistent naming. 1a4e9ea [Michael Armbrust] More comments. 2f4e7b9 [Michael Armbrust] Improve column pruning in the optimizer. --- .../sql/catalyst/optimizer/Optimizer.scala | 51 ++++++++++++++++++- .../plans/logical/basicOperators.scala | 2 +- 2 files changed, 51 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 37b23ba58289c..c0a09a16ac98d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -33,7 +33,56 @@ object Optimizer extends RuleExecutor[LogicalPlan] { Batch("Filter Pushdown", Once, CombineFilters, PushPredicateThroughProject, - PushPredicateThroughInnerJoin) :: Nil + PushPredicateThroughInnerJoin, + ColumnPruning) :: Nil +} + +/** + * Attempts to eliminate the reading of unneeded columns from the query plan using the following + * transformations: + * + * - Inserting Projections beneath the following operators: + * - Aggregate + * - Project <- Join + * - Collapse adjacent projections, performing alias substitution. + */ +object ColumnPruning extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty => + // Project away references that are not needed to calculate the required aggregates. + a.copy(child = Project(a.references.toSeq, child)) + + case Project(projectList, Join(left, right, joinType, condition)) => + // Collect the list of off references required either above or to evaluate the condition. + val allReferences: Set[Attribute] = + projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) + /** Applies a projection when the child is producing unnecessary attributes */ + def prunedChild(c: LogicalPlan) = + if ((allReferences.filter(c.outputSet.contains) -- c.outputSet).nonEmpty) { + Project(allReferences.filter(c.outputSet.contains).toSeq, c) + } else { + c + } + + Project(projectList, Join(prunedChild(left), prunedChild(right), joinType, condition)) + + case Project(projectList1, Project(projectList2, child)) => + // Create a map of Aliases to their values from the child projection. + // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). + val aliasMap = projectList2.collect { + case a @ Alias(e, _) => (a.toAttribute: Expression, a) + }.toMap + + // Substitute any attributes that are produced by the child projection, so that we safely + // eliminate it. + // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' + // TODO: Fix TransformBase to avoid the cast below. + val substitutedProjection = projectList1.map(_.transform { + case a if aliasMap.contains(a) => aliasMap(a) + }).asInstanceOf[Seq[NamedExpression]] + + Project(substitutedProjection, child) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index cfc0b0c3a8d98..397473e178867 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -127,7 +127,7 @@ case class Aggregate( extends UnaryNode { def output = aggregateExpressions.map(_.toAttribute) - def references = child.references + def references = (groupingExpressions ++ aggregateExpressions).flatMap(_.references).toSet } case class Limit(limit: Expression, child: LogicalPlan) extends UnaryNode { From 2c557837b4a12c644cc37bd00d02be04f3807637 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Thu, 10 Apr 2014 17:10:11 -0700 Subject: [PATCH 09/61] SPARK-1202 - Add a "cancel" button in the UI for stages Author: Sundeep Narravula Author: Sundeep Narravula Closes #246 from sundeepn/uikilljob and squashes the following commits: 5fdd0e2 [Sundeep Narravula] Fix test string f6fdff1 [Sundeep Narravula] Format fix; reduced line size to less than 100 chars d1daeb9 [Sundeep Narravula] Incorporating review comments. 8d97923 [Sundeep Narravula] Ability to kill jobs thru the UI. This behavior can be turned on be settings the following variable: spark.ui.killEnabled=true (default=false) Adding DAGScheduler event StageCancelled and corresponding handlers. Added cancellation reason to handlers. --- .../scala/org/apache/spark/SparkContext.scala | 10 ++++++ .../apache/spark/scheduler/DAGScheduler.scala | 32 ++++++++++++++++--- .../spark/scheduler/DAGSchedulerEvent.scala | 2 ++ .../scala/org/apache/spark/ui/SparkUI.scala | 1 + .../org/apache/spark/ui/jobs/IndexPage.scala | 14 +++++++- .../apache/spark/ui/jobs/JobProgressUI.scala | 1 + .../org/apache/spark/ui/jobs/StagePage.scala | 1 + .../org/apache/spark/ui/jobs/StageTable.scala | 29 +++++++++++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- docs/configuration.md | 7 ++++ 10 files changed, 87 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e6c9b7000d819..3bcc8ce2b25a6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1138,6 +1138,16 @@ class SparkContext(config: SparkConf) extends Logging { dagScheduler.cancelAllJobs() } + /** Cancel a given job if it's scheduled or running */ + private[spark] def cancelJob(jobId: Int) { + dagScheduler.cancelJob(jobId) + } + + /** Cancel a given stage and all jobs associated with it */ + private[spark] def cancelStage(stageId: Int) { + dagScheduler.cancelStage(stageId) + } + /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c41d6d75a1d49..c6cbf14e20069 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -511,6 +511,13 @@ class DAGScheduler( eventProcessActor ! AllJobsCancelled } + /** + * Cancel all jobs associated with a running or scheduled stage. + */ + def cancelStage(stageId: Int) { + eventProcessActor ! StageCancelled(stageId) + } + /** * Process one event retrieved from the event processing actor. * @@ -551,6 +558,9 @@ class DAGScheduler( submitStage(finalStage) } + case StageCancelled(stageId) => + handleStageCancellation(stageId) + case JobCancelled(jobId) => handleJobCancellation(jobId) @@ -560,11 +570,13 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(activeJob => groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach(handleJobCancellation) + jobIds.foreach(jobId => handleJobCancellation(jobId, + "as part of cancelled job group %s".format(groupId))) case AllJobsCancelled => // Cancel all running jobs. - runningStages.map(_.jobId).foreach(handleJobCancellation) + runningStages.map(_.jobId).foreach(jobId => handleJobCancellation(jobId, + "as part of cancellation of all jobs")) activeJobs.clear() // These should already be empty by this point, jobIdToActiveJob.clear() // but just in case we lost track of some jobs... @@ -991,11 +1003,23 @@ class DAGScheduler( } } - private def handleJobCancellation(jobId: Int) { + private def handleStageCancellation(stageId: Int) { + if (stageIdToJobIds.contains(stageId)) { + val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray + jobsThatUseStage.foreach(jobId => { + handleJobCancellation(jobId, "because Stage %s was cancelled".format(stageId)) + }) + } else { + logInfo("No active jobs to kill for Stage " + stageId) + } + } + + private def handleJobCancellation(jobId: Int, reason: String = "") { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled", None) + failJobAndIndependentStages(jobIdToActiveJob(jobId), + "Job %d cancelled %s".format(jobId, reason), None) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 293cfb65643a6..7367c08b5d324 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -44,6 +44,8 @@ private[scheduler] case class JobSubmitted( properties: Properties = null) extends DAGSchedulerEvent +private[scheduler] case class StageCancelled(stageId: Int) extends DAGSchedulerEvent + private[scheduler] case class JobCancelled(jobId: Int) extends DAGSchedulerEvent private[scheduler] case class JobGroupCancelled(groupId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index b8e6e15880bf5..dac11ec1cf52f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -46,6 +46,7 @@ private[spark] class SparkUI( val live = sc != null val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) + val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index f811aff616bcf..5da5d1f2a3f45 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -32,6 +32,7 @@ private[ui] class IndexPage(parent: JobProgressUI) { private val sc = parent.sc private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler + private val killEnabled = parent.killEnabled private def appName = parent.appName @@ -42,7 +43,18 @@ private[ui] class IndexPage(parent: JobProgressUI) { val failedStages = listener.failedStages.reverse.toSeq val now = System.currentTimeMillis() - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) + if (killEnabled) { + val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean + val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt + + if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { + sc.cancelStage(stageId) + } + } + + + val activeStagesTable = + new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index ad1a12cdc4e36..9de659d6c7393 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -32,6 +32,7 @@ private[ui] class JobProgressUI(parent: SparkUI) { val basePath = parent.basePath val live = parent.live val sc = parent.sc + val killEnabled = parent.killEnabled lazy val listener = _listener.get lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0bcbd7461cc5b..b6c3e3cf45163 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -30,6 +30,7 @@ import org.apache.spark.util.{Utils, Distribution} private[ui] class StagePage(parent: JobProgressUI) { private val basePath = parent.basePath private lazy val listener = parent.listener + private lazy val sc = parent.sc private def appName = parent.appName diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index ac61568af52d2..1e874ae4969f9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -27,7 +27,11 @@ import org.apache.spark.ui.{WebUI, UIUtils} import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { +private[ui] class StageTable( + stages: Seq[StageInfo], + parent: JobProgressUI, + killEnabled: Boolean = false) { + private val basePath = parent.basePath private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler @@ -71,15 +75,28 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { } - /** Render an HTML row that represents a stage */ - private def stageRow(s: StageInfo): Seq[Node] = { - val poolName = listener.stageIdToPool.get(s.stageId) + private def makeDescription(s: StageInfo): Seq[Node] = { val nameLink = {s.name} + val killLink = if (killEnabled) { + + + } val description = listener.stageIdToDescription.get(s.stageId) - .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) + .map(d =>
    {d}
    {nameLink} {killLink}
    ) + .getOrElse(
    {nameLink} {killLink}
    ) + + return description + } + + /** Render an HTML row that represents a stage */ + private def stageRow(s: StageInfo): Seq[Node] = { + val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { case Some(t) => WebUI.formatDate(new Date(t)) case None => "Unknown" @@ -118,7 +135,7 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { }} - {description} + {makeDescription(s)} {submissionTime} {formattedDuration} diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index a74724d785ad3..db4df1d1212ff 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -290,7 +290,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val rdd = makeRdd(1, Nil) val jobId = submit(rdd, Array(0)) cancel(jobId) - assert(failure.getMessage === s"Job $jobId cancelled") + assert(failure.getMessage === s"Job $jobId cancelled ") assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) diff --git a/docs/configuration.md b/docs/configuration.md index 9c602402f0635..f3bfd036f4164 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -190,6 +190,13 @@ Apart from these, the following properties are also available, and may be useful user that started the Spark job has view access. + + spark.ui.killEnabled + true + + Allows stages and corresponding jobs to be killed from the web ui. + + spark.shuffle.compress true From 5cd11d51c19321981a6234a7765c7a5be6913433 Mon Sep 17 00:00:00 2001 From: Ivan Wick Date: Thu, 10 Apr 2014 17:49:30 -0700 Subject: [PATCH 10/61] Set spark.executor.uri from environment variable (needed by Mesos) The Mesos backend uses this property when setting up a slave process. It is similarly set in the Scala repl (org.apache.spark.repl.SparkILoop), but I couldn't find any analogous for pyspark. Author: Ivan Wick This patch had conflicts when merged, resolved by Committer: Matei Zaharia Closes #311 from ivanwick/master and squashes the following commits: da0c3e4 [Ivan Wick] Set spark.executor.uri from environment variable (needed by Mesos) --- python/pyspark/shell.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 35e48276e3cb9..61613dbed8dce 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -29,6 +29,9 @@ # this is the equivalent of ADD_JARS add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None +if os.environ.get("SPARK_EXECUTOR_URI"): + SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) + sc = SparkContext(os.environ.get("MASTER", "local[*]"), "PySparkShell", pyFiles=add_files) print """Welcome to From 7b4203ab4c640f7875ae3536228ed4d791062017 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 10 Apr 2014 18:25:54 -0700 Subject: [PATCH 11/61] Add Spark v0.9.1 to ec2 launch script and use it as the default Mainly ported from branch-0.9. Author: Harvey Feng Closes #385 from harveyfeng/0.9.1-ec2 and squashes the following commits: 769ac2f [Harvey Feng] Add Spark v0.9.1 to ec2 launch script and use it as the default --- ec2/spark_ec2.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index d8840c94ac17c..31209a662bbe1 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -70,7 +70,7 @@ def parse_args(): "slaves across multiple (an additional $0.01/Gb for bandwidth" + "between zones applies)") parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") - parser.add_option("-v", "--spark-version", default="0.9.0", + parser.add_option("-v", "--spark-version", default="0.9.1", help="Version of Spark to use: 'X.Y.Z' or a specific git hash") parser.add_option("--spark-git-repo", default="https://github.com/apache/spark", @@ -157,7 +157,7 @@ def is_active(instance): # Return correct versions of Spark and Shark, given the supplied Spark version def get_spark_shark_version(opts): - spark_shark_map = {"0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0"} + spark_shark_map = {"0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0", "0.9.1": "0.9.1"} version = opts.spark_version.replace("v", "") if version not in spark_shark_map: print >> stderr, "Don't know about Spark version: %s" % version From 44f654eecd3c181f2aeaff3871acf7f00eacc6b9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 10 Apr 2014 20:43:56 -0700 Subject: [PATCH 12/61] SPARK-1202: Improvements to task killing in the UI. 1. Adds a separate endpoint for the killing logic that is outside of a page. 2. Narrows the scope of the killingEnabled tracking. 3. Some style improvements. Author: Patrick Wendell Closes #386 from pwendell/kill-link and squashes the following commits: 8efe02b [Patrick Wendell] Improvements to task killing in the UI. --- .../org/apache/spark/ui/static/webui.css | 9 +++++++++ .../scala/org/apache/spark/ui/JettyUtils.scala | 2 ++ .../scala/org/apache/spark/ui/SparkUI.scala | 5 ++--- .../org/apache/spark/ui/jobs/IndexPage.scala | 11 ----------- .../apache/spark/ui/jobs/JobProgressUI.scala | 17 ++++++++++++++++- .../org/apache/spark/ui/jobs/StageTable.scala | 16 +++++++++------- 6 files changed, 38 insertions(+), 22 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index fe54c34ffb1da..599c3ac9b57c0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -78,3 +78,12 @@ table.sortable thead { background-repeat: repeat-x; filter: progid:dximagetransform.microsoft.gradient(startColorstr='#FFA4EDFF', endColorstr='#FF94DDFF', GradientType=0); } + +span.kill-link { + margin-right: 2px; + color: gray; +} + +span.kill-link a { + color: gray; +} diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 9ce0398d010a8..dd0818e8ab01c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -104,10 +104,12 @@ private[spark] object JettyUtils extends Logging { def createRedirectHandler( srcPath: String, destPath: String, + beforeRedirect: HttpServletRequest => Unit = x => (), basePath: String = ""): ServletContextHandler = { val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + beforeRedirect(request) // Make sure we don't end up with "//" in the middle val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString response.sendRedirect(newUrl) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index dac11ec1cf52f..4c891d73afa87 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI( val sc: SparkContext, - conf: SparkConf, + val conf: SparkConf, val listenerBus: SparkListenerBus, var appName: String, val basePath: String = "") @@ -46,7 +46,6 @@ private[spark] class SparkUI( val live = sc != null val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) @@ -70,7 +69,7 @@ private[spark] class SparkUI( metricsServletHandlers ++ Seq[ServletContextHandler] ( createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"), - createRedirectHandler("/", "/stages", basePath) + createRedirectHandler("/", "/stages", basePath = basePath) ) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 5da5d1f2a3f45..8619a31380f1e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -32,7 +32,6 @@ private[ui] class IndexPage(parent: JobProgressUI) { private val sc = parent.sc private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler - private val killEnabled = parent.killEnabled private def appName = parent.appName @@ -43,16 +42,6 @@ private[ui] class IndexPage(parent: JobProgressUI) { val failedStages = listener.failedStages.reverse.toSeq val now = System.currentTimeMillis() - if (killEnabled) { - val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean - val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt - - if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { - sc.cancelStage(stageId) - } - } - - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) val completedStagesTable = diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 9de659d6c7393..30e3f35f2182b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -32,7 +32,7 @@ private[ui] class JobProgressUI(parent: SparkUI) { val basePath = parent.basePath val live = parent.live val sc = parent.sc - val killEnabled = parent.killEnabled + val killEnabled = parent.conf.getBoolean("spark.ui.killEnabled", true) lazy val listener = _listener.get lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) @@ -51,7 +51,22 @@ private[ui] class JobProgressUI(parent: SparkUI) { def formatDuration(ms: Long) = Utils.msDurationToString(ms) + private def handleKillRequest(request: HttpServletRequest) = { + if (killEnabled) { + val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean + val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt + if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { + sc.cancelStage(stageId) + } + // Do a quick pause here to give Spark time to kill the stage so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } + } + def getHandlers = Seq[ServletContextHandler]( + createRedirectHandler("/stages/stage/kill", "/stages", handleKillRequest), createServletHandler("/stages/stage", (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), createServletHandler("/stages/pool", diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 1e874ae4969f9..e419fae5a6589 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -76,20 +76,22 @@ private[ui] class StageTable( } private def makeDescription(s: StageInfo): Seq[Node] = { + // scalastyle:off + val killLink = if (killEnabled) { + + (kill) + + } + // scalastyle:on + val nameLink = {s.name} - val killLink = if (killEnabled) { - - } val description = listener.stageIdToDescription.get(s.stageId) .map(d =>
    {d}
    {nameLink} {killLink}
    ) - .getOrElse(
    {nameLink} {killLink}
    ) + .getOrElse(
    {killLink}{nameLink}
    ) return description } From 446bb3417a2855a194d49acc0ac316a021eced9d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 11 Apr 2014 13:17:48 +0530 Subject: [PATCH 13/61] SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken Author: Thomas Graves Closes #344 from tgravescs/SPARK-1417 and squashes the following commits: c450b5f [Thomas Graves] fix test e1c1d7e [Thomas Graves] add missing $ to appUIAddress e982ddb [Thomas Graves] use appUIHostPort in appUIAddress 0803ec2 [Thomas Graves] Review comment updates - remove extra newline, simplify assert in test 658a8ec [Thomas Graves] Add a appUIHostPort routine 0614208 [Thomas Graves] Fix test 2a6b1b7 [Thomas Graves] SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken --- .../scala/org/apache/spark/ui/SparkUI.scala | 7 +++- .../scala/org/apache/spark/SparkUISuite.scala | 35 +++++++++++++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- 4 files changed, 43 insertions(+), 3 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/SparkUISuite.scala diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 4c891d73afa87..7fa4fd3149eb6 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -113,7 +113,12 @@ private[spark] class SparkUI( logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = "http://" + publicHost + ":" + boundPort + /** + * Return the application UI host:port. This does not include the scheme (http://). + */ + private[spark] def appUIHostPort = publicHost + ":" + boundPort + + private[spark] def appUIAddress = s"http://$appUIHostPort" } diff --git a/core/src/test/scala/org/apache/spark/SparkUISuite.scala b/core/src/test/scala/org/apache/spark/SparkUISuite.scala new file mode 100644 index 0000000000000..d0d119c15081d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SparkUISuite.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.net.URI + +import org.scalatest.FunSuite + +class SparkUISuite extends FunSuite with SharedSparkContext { + + test("verify appUIAddress contains the scheme") { + val uiAddress = sc.ui.appUIAddress + assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + } + + test("verify appUIAddress contains the port") { + val splitUIAddress = sc.ui.appUIAddress.split(':') + assert(splitUIAddress(2).toInt == sc.ui.boundPort) + } +} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 910484ed5432a..67ec95c8fc04f 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -234,7 +234,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, assert(sparkContext != null || count >= numTries) if (null != sparkContext) { - uiAddress = sparkContext.ui.appUIAddress + uiAddress = sparkContext.ui.appUIHostPort this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, resourceManager, diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index c8a4d2e647cbd..61af0f9ac5ca0 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -220,7 +220,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, assert(sparkContext != null || numTries >= maxNumTries) if (sparkContext != null) { - uiAddress = sparkContext.ui.appUIAddress + uiAddress = sparkContext.ui.appUIHostPort this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, amClient, From 98225a6effd077a1b97c7e485d45ffd89b2c5b7f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 11 Apr 2014 10:45:27 -0700 Subject: [PATCH 14/61] Some clean up in build/docs (a) Deleted an outdated line from the docs (b) Removed a work around that is no longer necessary given the mesos version bump. Author: Patrick Wendell Closes #382 from pwendell/maven-clean and squashes the following commits: f0447fa [Patrick Wendell] Minor doc clean-up --- docs/index.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/index.md b/docs/index.md index 7a13fa9a9a2b6..89ec5b05488a9 100644 --- a/docs/index.md +++ b/docs/index.md @@ -67,8 +67,6 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`. -For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.html). This is needed because Hadoop 2.2 has non backwards compatible API changes. - # Where to Go from Here **Programming guides:** From f5ace8da34c58d1005c7c377cfe3df21102c1dd6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 11 Apr 2014 12:06:13 -0700 Subject: [PATCH 15/61] [SPARK-1225, 1241] [MLLIB] Add AreaUnderCurve and BinaryClassificationMetrics This PR implements a generic version of `AreaUnderCurve` using the `RDD.sliding` implementation from https://github.com/apache/spark/pull/136 . It also contains refactoring of https://github.com/apache/spark/pull/160 for binary classification evaluation. Author: Xiangrui Meng Closes #364 from mengxr/auc and squashes the following commits: a05941d [Xiangrui Meng] replace TP/FP/TN/FN by their full names 3f42e98 [Xiangrui Meng] add (0, 0), (1, 1) to roc, and (0, 1) to pr fb4b6d2 [Xiangrui Meng] rename Evaluator to Metrics and add more metrics b1b7dab [Xiangrui Meng] fix code styles 9dc3518 [Xiangrui Meng] add tests for BinaryClassificationEvaluator ca31da5 [Xiangrui Meng] remove PredictionAndResponse 3d71525 [Xiangrui Meng] move binary evalution classes to evaluation.binary 8f78958 [Xiangrui Meng] add PredictionAndResponse dda82d5 [Xiangrui Meng] add confusion matrix aa7e278 [Xiangrui Meng] add initial version of binary classification evaluator 221ebce [Xiangrui Meng] add a new test to sliding a920865 [Xiangrui Meng] Merge branch 'sliding' into auc a9b250a [Xiangrui Meng] move sliding to mllib cab9a52 [Xiangrui Meng] use last for the last element db6cb30 [Xiangrui Meng] remove unnecessary toSeq 9916202 [Xiangrui Meng] change RDD.sliding return type to RDD[Seq[T]] 284d991 [Xiangrui Meng] change SlidedRDD to SlidingRDD c1c6c22 [Xiangrui Meng] add AreaUnderCurve 65461b2 [Xiangrui Meng] Merge branch 'sliding' into auc 5ee6001 [Xiangrui Meng] add TODO d2a600d [Xiangrui Meng] add sliding to rdd --- .../mllib/evaluation/AreaUnderCurve.scala | 62 ++++++ .../BinaryClassificationMetricComputers.scala | 57 +++++ .../binary/BinaryClassificationMetrics.scala | 204 ++++++++++++++++++ .../binary/BinaryConfusionMatrix.scala | 41 ++++ .../apache/spark/mllib/rdd/RDDFunctions.scala | 53 +++++ .../apache/spark/mllib/rdd/SlidingRDD.scala | 104 +++++++++ .../evaluation/AreaUnderCurveSuite.scala | 46 ++++ .../BinaryClassificationMetricsSuite.scala | 55 +++++ .../spark/mllib/rdd/RDDFunctionsSuite.scala | 49 +++++ 9 files changed, 671 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala new file mode 100644 index 0000000000000..7858ec602483f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.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 org.apache.spark.mllib.evaluation + +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.rdd.RDDFunctions._ + +/** + * Computes the area under the curve (AUC) using the trapezoidal rule. + */ +private[evaluation] object AreaUnderCurve { + + /** + * Uses the trapezoidal rule to compute the area under the line connecting the two input points. + * @param points two 2D points stored in Seq + */ + private def trapezoid(points: Seq[(Double, Double)]): Double = { + require(points.length == 2) + val x = points.head + val y = points.last + (y._1 - x._1) * (y._2 + x._2) / 2.0 + } + + /** + * Returns the area under the given curve. + * + * @param curve a RDD of ordered 2D points stored in pairs representing a curve + */ + def of(curve: RDD[(Double, Double)]): Double = { + curve.sliding(2).aggregate(0.0)( + seqOp = (auc: Double, points: Seq[(Double, Double)]) => auc + trapezoid(points), + combOp = _ + _ + ) + } + + /** + * Returns the area under the given curve. + * + * @param curve an iterator over ordered 2D points stored in pairs representing a curve + */ + def of(curve: Iterable[(Double, Double)]): Double = { + curve.toIterator.sliding(2).withPartial(false).aggregate(0.0)( + seqop = (auc: Double, points: Seq[(Double, Double)]) => auc + trapezoid(points), + combop = _ + _ + ) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala new file mode 100644 index 0000000000000..562663ad36b40 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +/** + * Trait for a binary classification evaluation metric computer. + */ +private[evaluation] trait BinaryClassificationMetricComputer extends Serializable { + def apply(c: BinaryConfusionMatrix): Double +} + +/** Precision. */ +private[evaluation] object Precision extends BinaryClassificationMetricComputer { + override def apply(c: BinaryConfusionMatrix): Double = + c.numTruePositives.toDouble / (c.numTruePositives + c.numFalsePositives) +} + +/** False positive rate. */ +private[evaluation] object FalsePositiveRate extends BinaryClassificationMetricComputer { + override def apply(c: BinaryConfusionMatrix): Double = + c.numFalsePositives.toDouble / c.numNegatives +} + +/** Recall. */ +private[evaluation] object Recall extends BinaryClassificationMetricComputer { + override def apply(c: BinaryConfusionMatrix): Double = + c.numTruePositives.toDouble / c.numPositives +} + +/** + * F-Measure. + * @param beta the beta constant in F-Measure + * @see http://en.wikipedia.org/wiki/F1_score + */ +private[evaluation] case class FMeasure(beta: Double) extends BinaryClassificationMetricComputer { + private val beta2 = beta * beta + override def apply(c: BinaryConfusionMatrix): Double = { + val precision = Precision(c) + val recall = Recall(c) + (1.0 + beta2) * (precision * recall) / (beta2 * precision + recall) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala new file mode 100644 index 0000000000000..ed7b0fc943367 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +import org.apache.spark.rdd.{UnionRDD, RDD} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.evaluation.AreaUnderCurve +import org.apache.spark.Logging + +/** + * Implementation of [[org.apache.spark.mllib.evaluation.binary.BinaryConfusionMatrix]]. + * + * @param count label counter for labels with scores greater than or equal to the current score + * @param totalCount label counter for all labels + */ +private case class BinaryConfusionMatrixImpl( + count: LabelCounter, + totalCount: LabelCounter) extends BinaryConfusionMatrix with Serializable { + + /** number of true positives */ + override def numTruePositives: Long = count.numPositives + + /** number of false positives */ + override def numFalsePositives: Long = count.numNegatives + + /** number of false negatives */ + override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives + + /** number of true negatives */ + override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives + + /** number of positives */ + override def numPositives: Long = totalCount.numPositives + + /** number of negatives */ + override def numNegatives: Long = totalCount.numNegatives +} + +/** + * Evaluator for binary classification. + * + * @param scoreAndLabels an RDD of (score, label) pairs. + */ +class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) + extends Serializable with Logging { + + private lazy val ( + cumulativeCounts: RDD[(Double, LabelCounter)], + confusions: RDD[(Double, BinaryConfusionMatrix)]) = { + // Create a bin for each distinct score value, count positives and negatives within each bin, + // and then sort by score values in descending order. + val counts = scoreAndLabels.combineByKey( + createCombiner = (label: Double) => new LabelCounter(0L, 0L) += label, + mergeValue = (c: LabelCounter, label: Double) => c += label, + mergeCombiners = (c1: LabelCounter, c2: LabelCounter) => c1 += c2 + ).sortByKey(ascending = false) + val agg = counts.values.mapPartitions({ iter => + val agg = new LabelCounter() + iter.foreach(agg += _) + Iterator(agg) + }, preservesPartitioning = true).collect() + val partitionwiseCumulativeCounts = + agg.scanLeft(new LabelCounter())((agg: LabelCounter, c: LabelCounter) => agg.clone() += c) + val totalCount = partitionwiseCumulativeCounts.last + logInfo(s"Total counts: $totalCount") + val cumulativeCounts = counts.mapPartitionsWithIndex( + (index: Int, iter: Iterator[(Double, LabelCounter)]) => { + val cumCount = partitionwiseCumulativeCounts(index) + iter.map { case (score, c) => + cumCount += c + (score, cumCount.clone()) + } + }, preservesPartitioning = true) + cumulativeCounts.persist() + val confusions = cumulativeCounts.map { case (score, cumCount) => + (score, BinaryConfusionMatrixImpl(cumCount, totalCount).asInstanceOf[BinaryConfusionMatrix]) + } + (cumulativeCounts, confusions) + } + + /** Unpersist intermediate RDDs used in the computation. */ + def unpersist() { + cumulativeCounts.unpersist() + } + + /** Returns thresholds in descending order. */ + def thresholds(): RDD[Double] = cumulativeCounts.map(_._1) + + /** + * Returns the receiver operating characteristic (ROC) curve, + * which is an RDD of (false positive rate, true positive rate) + * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. + * @see http://en.wikipedia.org/wiki/Receiver_operating_characteristic + */ + def roc(): RDD[(Double, Double)] = { + val rocCurve = createCurve(FalsePositiveRate, Recall) + val sc = confusions.context + val first = sc.makeRDD(Seq((0.0, 0.0)), 1) + val last = sc.makeRDD(Seq((1.0, 1.0)), 1) + new UnionRDD[(Double, Double)](sc, Seq(first, rocCurve, last)) + } + + /** + * Computes the area under the receiver operating characteristic (ROC) curve. + */ + def areaUnderROC(): Double = AreaUnderCurve.of(roc()) + + /** + * Returns the precision-recall curve, which is an RDD of (recall, precision), + * NOT (precision, recall), with (0.0, 1.0) prepended to it. + * @see http://en.wikipedia.org/wiki/Precision_and_recall + */ + def pr(): RDD[(Double, Double)] = { + val prCurve = createCurve(Recall, Precision) + val sc = confusions.context + val first = sc.makeRDD(Seq((0.0, 1.0)), 1) + first.union(prCurve) + } + + /** + * Computes the area under the precision-recall curve. + */ + def areaUnderPR(): Double = AreaUnderCurve.of(pr()) + + /** + * Returns the (threshold, F-Measure) curve. + * @param beta the beta factor in F-Measure computation. + * @return an RDD of (threshold, F-Measure) pairs. + * @see http://en.wikipedia.org/wiki/F1_score + */ + def fMeasureByThreshold(beta: Double): RDD[(Double, Double)] = createCurve(FMeasure(beta)) + + /** Returns the (threshold, F-Measure) curve with beta = 1.0. */ + def fMeasureByThreshold(): RDD[(Double, Double)] = fMeasureByThreshold(1.0) + + /** Returns the (threshold, precision) curve. */ + def precisionByThreshold(): RDD[(Double, Double)] = createCurve(Precision) + + /** Returns the (threshold, recall) curve. */ + def recallByThreshold(): RDD[(Double, Double)] = createCurve(Recall) + + /** Creates a curve of (threshold, metric). */ + private def createCurve(y: BinaryClassificationMetricComputer): RDD[(Double, Double)] = { + confusions.map { case (s, c) => + (s, y(c)) + } + } + + /** Creates a curve of (metricX, metricY). */ + private def createCurve( + x: BinaryClassificationMetricComputer, + y: BinaryClassificationMetricComputer): RDD[(Double, Double)] = { + confusions.map { case (_, c) => + (x(c), y(c)) + } + } +} + +/** + * A counter for positives and negatives. + * + * @param numPositives number of positive labels + * @param numNegatives number of negative labels + */ +private class LabelCounter( + var numPositives: Long = 0L, + var numNegatives: Long = 0L) extends Serializable { + + /** Processes a label. */ + def +=(label: Double): LabelCounter = { + // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle + // -1.0 for negative as well. + if (label > 0.5) numPositives += 1L else numNegatives += 1L + this + } + + /** Merges another counter. */ + def +=(other: LabelCounter): LabelCounter = { + numPositives += other.numPositives + numNegatives += other.numNegatives + this + } + + override def clone: LabelCounter = { + new LabelCounter(numPositives, numNegatives) + } + + override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}" +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala new file mode 100644 index 0000000000000..75a75b216002a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +/** + * Trait for a binary confusion matrix. + */ +private[evaluation] trait BinaryConfusionMatrix { + /** number of true positives */ + def numTruePositives: Long + + /** number of false positives */ + def numFalsePositives: Long + + /** number of false negatives */ + def numFalseNegatives: Long + + /** number of true negatives */ + def numTrueNegatives: Long + + /** number of positives */ + def numPositives: Long = numTruePositives + numFalseNegatives + + /** number of negatives */ + def numNegatives: Long = numFalsePositives + numTrueNegatives +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala new file mode 100644 index 0000000000000..873de871fd884 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.rdd + +import scala.reflect.ClassTag + +import org.apache.spark.rdd.RDD + +/** + * Machine learning specific RDD functions. + */ +private[mllib] +class RDDFunctions[T: ClassTag](self: RDD[T]) { + + /** + * Returns a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding + * window over them. The ordering is first based on the partition index and then the ordering of + * items within each partition. This is similar to sliding in Scala collections, except that it + * becomes an empty RDD if the window size is greater than the total number of items. It needs to + * trigger a Spark job if the parent RDD has more than one partitions and the window size is + * greater than 1. + */ + def sliding(windowSize: Int): RDD[Seq[T]] = { + require(windowSize > 0, s"Sliding window size must be positive, but got $windowSize.") + if (windowSize == 1) { + self.map(Seq(_)) + } else { + new SlidingRDD[T](self, windowSize) + } + } +} + +private[mllib] +object RDDFunctions { + + /** Implicit conversion from an RDD to RDDFunctions. */ + implicit def fromRDD[T: ClassTag](rdd: RDD[T]) = new RDDFunctions[T](rdd) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala new file mode 100644 index 0000000000000..dd80782c0f001 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.rdd + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.{TaskContext, Partition} +import org.apache.spark.rdd.RDD + +private[mllib] +class SlidingRDDPartition[T](val idx: Int, val prev: Partition, val tail: Seq[T]) + extends Partition with Serializable { + override val index: Int = idx +} + +/** + * Represents a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding + * window over them. The ordering is first based on the partition index and then the ordering of + * items within each partition. This is similar to sliding in Scala collections, except that it + * becomes an empty RDD if the window size is greater than the total number of items. It needs to + * trigger a Spark job if the parent RDD has more than one partitions. To make this operation + * efficient, the number of items per partition should be larger than the window size and the + * window size should be small, e.g., 2. + * + * @param parent the parent RDD + * @param windowSize the window size, must be greater than 1 + * + * @see [[org.apache.spark.mllib.rdd.RDDFunctions#sliding]] + */ +private[mllib] +class SlidingRDD[T: ClassTag](@transient val parent: RDD[T], val windowSize: Int) + extends RDD[Seq[T]](parent) { + + require(windowSize > 1, s"Window size must be greater than 1, but got $windowSize.") + + override def compute(split: Partition, context: TaskContext): Iterator[Seq[T]] = { + val part = split.asInstanceOf[SlidingRDDPartition[T]] + (firstParent[T].iterator(part.prev, context) ++ part.tail) + .sliding(windowSize) + .withPartial(false) + } + + override def getPreferredLocations(split: Partition): Seq[String] = + firstParent[T].preferredLocations(split.asInstanceOf[SlidingRDDPartition[T]].prev) + + override def getPartitions: Array[Partition] = { + val parentPartitions = parent.partitions + val n = parentPartitions.size + if (n == 0) { + Array.empty + } else if (n == 1) { + Array(new SlidingRDDPartition[T](0, parentPartitions(0), Seq.empty)) + } else { + val n1 = n - 1 + val w1 = windowSize - 1 + // Get the first w1 items of each partition, starting from the second partition. + val nextHeads = + parent.context.runJob(parent, (iter: Iterator[T]) => iter.take(w1).toArray, 1 until n, true) + val partitions = mutable.ArrayBuffer[SlidingRDDPartition[T]]() + var i = 0 + var partitionIndex = 0 + while (i < n1) { + var j = i + val tail = mutable.ListBuffer[T]() + // Keep appending to the current tail until appended a head of size w1. + while (j < n1 && nextHeads(j).size < w1) { + tail ++= nextHeads(j) + j += 1 + } + if (j < n1) { + tail ++= nextHeads(j) + j += 1 + } + partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(i), tail) + partitionIndex += 1 + // Skip appended heads. + i = j + } + // If the head of last partition has size w1, we also need to add this partition. + if (nextHeads.last.size == w1) { + partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(n1), Seq.empty) + } + partitions.toArray + } + } + + // TODO: Override methods such as aggregate, which only requires one Spark job. +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala new file mode 100644 index 0000000000000..1c9844f289fe0 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext + +class AreaUnderCurveSuite extends FunSuite with LocalSparkContext { + test("auc computation") { + val curve = Seq((0.0, 0.0), (1.0, 1.0), (2.0, 3.0), (3.0, 0.0)) + val auc = 4.0 + assert(AreaUnderCurve.of(curve) === auc) + val rddCurve = sc.parallelize(curve, 2) + assert(AreaUnderCurve.of(rddCurve) == auc) + } + + test("auc of an empty curve") { + val curve = Seq.empty[(Double, Double)] + assert(AreaUnderCurve.of(curve) === 0.0) + val rddCurve = sc.parallelize(curve, 2) + assert(AreaUnderCurve.of(rddCurve) === 0.0) + } + + test("auc of a curve with a single point") { + val curve = Seq((1.0, 1.0)) + assert(AreaUnderCurve.of(curve) === 0.0) + val rddCurve = sc.parallelize(curve, 2) + assert(AreaUnderCurve.of(rddCurve) === 0.0) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala new file mode 100644 index 0000000000000..173fdaefab3da --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.evaluation.AreaUnderCurve + +class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { + test("binary evaluation metrics") { + val scoreAndLabels = sc.parallelize( + Seq((0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)), 2) + val metrics = new BinaryClassificationMetrics(scoreAndLabels) + val threshold = Seq(0.8, 0.6, 0.4, 0.1) + val numTruePositives = Seq(1, 3, 3, 4) + val numFalsePositives = Seq(0, 1, 2, 3) + val numPositives = 4 + val numNegatives = 3 + val precision = numTruePositives.zip(numFalsePositives).map { case (t, f) => + t.toDouble / (t + f) + } + val recall = numTruePositives.map(t => t.toDouble / numPositives) + val fpr = numFalsePositives.map(f => f.toDouble / numNegatives) + val rocCurve = Seq((0.0, 0.0)) ++ fpr.zip(recall) ++ Seq((1.0, 1.0)) + val pr = recall.zip(precision) + val prCurve = Seq((0.0, 1.0)) ++ pr + val f1 = pr.map { case (r, p) => 2.0 * (p * r) / (p + r) } + val f2 = pr.map { case (r, p) => 5.0 * (p * r) / (4.0 * p + r)} + assert(metrics.thresholds().collect().toSeq === threshold) + assert(metrics.roc().collect().toSeq === rocCurve) + assert(metrics.areaUnderROC() === AreaUnderCurve.of(rocCurve)) + assert(metrics.pr().collect().toSeq === prCurve) + assert(metrics.areaUnderPR() === AreaUnderCurve.of(prCurve)) + assert(metrics.fMeasureByThreshold().collect().toSeq === threshold.zip(f1)) + assert(metrics.fMeasureByThreshold(2.0).collect().toSeq === threshold.zip(f2)) + assert(metrics.precisionByThreshold().collect().toSeq === threshold.zip(precision)) + assert(metrics.recallByThreshold().collect().toSeq === threshold.zip(recall)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala new file mode 100644 index 0000000000000..3f3b10dfff35e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.rdd + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.rdd.RDDFunctions._ + +class RDDFunctionsSuite extends FunSuite with LocalSparkContext { + + test("sliding") { + val data = 0 until 6 + for (numPartitions <- 1 to 8) { + val rdd = sc.parallelize(data, numPartitions) + for (windowSize <- 1 to 6) { + val sliding = rdd.sliding(windowSize).collect().map(_.toList).toList + val expected = data.sliding(windowSize).map(_.toList).toList + assert(sliding === expected) + } + assert(rdd.sliding(7).collect().isEmpty, + "Should return an empty RDD if the window size is greater than the number of items.") + } + } + + test("sliding with empty partitions") { + val data = Seq(Seq(1, 2, 3), Seq.empty[Int], Seq(4), Seq.empty[Int], Seq(5, 6, 7)) + val rdd = sc.parallelize(data, data.length).flatMap(s => s) + assert(rdd.partitions.size === data.length) + val sliding = rdd.sliding(3) + val expected = data.flatMap(x => x).sliding(3).toList + assert(sliding.collect().toList === expected) + } +} From 6a0f8e35ce7595c4ece11fe04133fd44ffbe5b06 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 11 Apr 2014 13:23:21 -0700 Subject: [PATCH 16/61] HOTFIX: Ignore python metastore files in RAT checks. This was causing some errors with pull request tests. Author: Patrick Wendell Closes #393 from pwendell/hotfix and squashes the following commits: 6201dd3 [Patrick Wendell] HOTFIX: Ignore python metastore files in RAT checks. --- .rat-excludes | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.rat-excludes b/.rat-excludes index a2b5665a0be26..8954330bd10a7 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -39,4 +39,6 @@ work .*\.q golden test.out/* -.*iml \ No newline at end of file +.*iml +python/metastore/service.properties +python/metastore/db.lck From 7038b00be9c84a4d92f9d95ff3d75fae47d57d87 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 11 Apr 2014 19:41:40 -0700 Subject: [PATCH 17/61] [FIX] make coalesce test deterministic in RDDSuite Make coalesce test deterministic by setting pre-defined seeds. (Saw random failures in other PRs.) Author: Xiangrui Meng Closes #387 from mengxr/fix-random and squashes the following commits: 59bc16f [Xiangrui Meng] make coalesce test deterministic in RDDSuite --- .../scala/org/apache/spark/rdd/RDDSuite.scala | 61 ++++++++++--------- 1 file changed, 33 insertions(+), 28 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 25973348a7837..1901330d8b188 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -274,37 +274,42 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("coalesced RDDs with locality, large scale (10K partitions)") { // large scale experiment import collection.mutable - val rnd = scala.util.Random val partitions = 10000 val numMachines = 50 val machines = mutable.ListBuffer[String]() - (1 to numMachines).foreach(machines += "m"+_) - - val blocks = (1 to partitions).map(i => - { (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) } ) - - val data2 = sc.makeRDD(blocks) - val coalesced2 = data2.coalesce(numMachines*2) - - // test that you get over 90% locality in each group - val minLocality = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) - assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.0).toInt + "%") - - // test that the groups are load balanced with 100 +/- 20 elements in each - val maxImbalance = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) - .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) - assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) - - val data3 = sc.makeRDD(blocks).map(i => i*2) // derived RDD to test *current* pref locs - val coalesced3 = data3.coalesce(numMachines*2) - val minLocality2 = coalesced3.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) - assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + - (minLocality2*100.0).toInt + "%") + (1 to numMachines).foreach(machines += "m" + _) + val rnd = scala.util.Random + for (seed <- 1 to 5) { + rnd.setSeed(seed) + + val blocks = (1 to partitions).map { i => + (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) + } + + val data2 = sc.makeRDD(blocks) + val coalesced2 = data2.coalesce(numMachines * 2) + + // test that you get over 90% locality in each group + val minLocality = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.0)((perc, loc) => math.min(perc, loc)) + assert(minLocality >= 0.90, "Expected 90% locality but got " + + (minLocality * 100.0).toInt + "%") + + // test that the groups are load balanced with 100 +/- 20 elements in each + val maxImbalance = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) + .foldLeft(0)((dev, curr) => math.max(math.abs(100 - curr), dev)) + assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) + + val data3 = sc.makeRDD(blocks).map(i => i * 2) // derived RDD to test *current* pref locs + val coalesced3 = data3.coalesce(numMachines * 2) + val minLocality2 = coalesced3.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.0)((perc, loc) => math.min(perc, loc)) + assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + + (minLocality2 * 100.0).toInt + "%") + } } test("zipped RDDs") { From fdfb45e691946f3153d6c696bec6d7f3e391e301 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Fri, 11 Apr 2014 19:43:22 -0700 Subject: [PATCH 18/61] [WIP] [SPARK-1328] Add vector statistics As with the new vector system in MLlib, we find that it is good to add some new APIs to precess the `RDD[Vector]`. Beside, the former implementation of `computeStat` is not stable which could loss precision, and has the possibility to cause `Nan` in scientific computing, just as said in the [SPARK-1328](https://spark-project.atlassian.net/browse/SPARK-1328). APIs contain: * rowMeans(): RDD[Double] * rowNorm2(): RDD[Double] * rowSDs(): RDD[Double] * colMeans(): Vector * colMeans(size: Int): Vector * colNorm2(): Vector * colNorm2(size: Int): Vector * colSDs(): Vector * colSDs(size: Int): Vector * maxOption((Vector, Vector) => Boolean): Option[Vector] * minOption((Vector, Vector) => Boolean): Option[Vector] * rowShrink(): RDD[Vector] * colShrink(): RDD[Vector] This is working in process now, and some more APIs will add to `LabeledPoint`. Moreover, the implicit declaration will move from `MLUtils` to `MLContext` later. Author: Xusen Yin Author: Xiangrui Meng Closes #268 from yinxusen/vector-statistics and squashes the following commits: d61363f [Xusen Yin] rebase to latest master 16ae684 [Xusen Yin] fix minor error and remove useless method 10cf5d3 [Xusen Yin] refine some return type b064714 [Xusen Yin] remove computeStat in MLUtils cbbefdb [Xiangrui Meng] update multivariate statistical summary interface and clean tests 4eaf28a [Xusen Yin] merge VectorRDDStatistics into RowMatrix 48ee053 [Xusen Yin] fix minor error e624f93 [Xusen Yin] fix scala style error 1fba230 [Xusen Yin] merge while loop together 69e1f37 [Xusen Yin] remove lazy eval, and minor memory footprint 548e9de [Xusen Yin] minor revision 86522c4 [Xusen Yin] add comments on functions dc77e38 [Xusen Yin] test sparse vector RDD 18cf072 [Xusen Yin] change def to lazy val to make sure that the computations in function be evaluated only once f7a3ca2 [Xusen Yin] fix the corner case of maxmin 967d041 [Xusen Yin] full revision with Aggregator class 138300c [Xusen Yin] add new Aggregator class 1376ff4 [Xusen Yin] rename variables and adjust code 4a5c38d [Xusen Yin] add scala doc, refine code and comments 036b7a5 [Xusen Yin] fix the bug of Nan occur f6e8e9a [Xusen Yin] add sparse vectors test 4cfbadf [Xusen Yin] fix bug of min max 4e4fbd1 [Xusen Yin] separate seqop and combop out as independent functions a6d5a2e [Xusen Yin] rewrite for only computing non-zero elements 3980287 [Xusen Yin] rename variables 62a2c3e [Xusen Yin] use axpy and in-place if possible 9a75ebd [Xusen Yin] add case class to wrap return values d816ac7 [Xusen Yin] remove useless APIs c4651bb [Xusen Yin] remove row-wise APIs and refine code 1338ea1 [Xusen Yin] all-in-one version test passed cc65810 [Xusen Yin] add parallel mean and variance 9af2e95 [Xusen Yin] refine the code style ad6c82d [Xusen Yin] add shrink test e09d5d2 [Xusen Yin] add scala docs and refine shrink method 8ef3377 [Xusen Yin] pass all tests 28cf060 [Xusen Yin] fix error of column means 54b19ab [Xusen Yin] add new API to shrink RDD[Vector] 8c6c0e1 [Xusen Yin] add basic statistics --- .../mllib/linalg/distributed/RowMatrix.scala | 165 +++++++++++++++++- .../stat/MultivariateStatisticalSummary.scala | 56 ++++++ .../org/apache/spark/mllib/util/MLUtils.scala | 57 +----- .../linalg/distributed/RowMatrixSuite.scala | 15 ++ .../spark/mllib/util/MLUtilsSuite.scala | 13 -- 5 files changed, 230 insertions(+), 76 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index f65f43dd3007b..0c0afcd9ec0d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg.distributed import java.util -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} +import breeze.linalg.{Vector => BV, DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -27,6 +27,138 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.Logging +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary + +/** + * Column statistics aggregator implementing + * [[org.apache.spark.mllib.stat.MultivariateStatisticalSummary]] + * together with add() and merge() function. + * A numerically stable algorithm is implemented to compute sample mean and variance: + *[[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance variance-wiki]]. + * Zero elements (including explicit zero values) are skipped when calling add() and merge(), + * to have time complexity O(nnz) instead of O(n) for each column. + */ +private class ColumnStatisticsAggregator(private val n: Int) + extends MultivariateStatisticalSummary with Serializable { + + private val currMean: BDV[Double] = BDV.zeros[Double](n) + private val currM2n: BDV[Double] = BDV.zeros[Double](n) + private var totalCnt = 0.0 + private val nnz: BDV[Double] = BDV.zeros[Double](n) + private val currMax: BDV[Double] = BDV.fill(n)(Double.MinValue) + private val currMin: BDV[Double] = BDV.fill(n)(Double.MaxValue) + + override def mean: Vector = { + val realMean = BDV.zeros[Double](n) + var i = 0 + while (i < n) { + realMean(i) = currMean(i) * nnz(i) / totalCnt + i += 1 + } + Vectors.fromBreeze(realMean) + } + + override def variance: Vector = { + val realVariance = BDV.zeros[Double](n) + + val denominator = totalCnt - 1.0 + + // Sample variance is computed, if the denominator is less than 0, the variance is just 0. + if (denominator > 0.0) { + val deltaMean = currMean + var i = 0 + while (i < currM2n.size) { + realVariance(i) = + currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * (totalCnt - nnz(i)) / totalCnt + realVariance(i) /= denominator + i += 1 + } + } + + Vectors.fromBreeze(realVariance) + } + + override def count: Long = totalCnt.toLong + + override def numNonzeros: Vector = Vectors.fromBreeze(nnz) + + override def max: Vector = { + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMax(i) < 0.0)) currMax(i) = 0.0 + i += 1 + } + Vectors.fromBreeze(currMax) + } + + override def min: Vector = { + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMin(i) > 0.0)) currMin(i) = 0.0 + i += 1 + } + Vectors.fromBreeze(currMin) + } + + /** + * Aggregates a row. + */ + def add(currData: BV[Double]): this.type = { + currData.activeIterator.foreach { + case (_, 0.0) => // Skip explicit zero elements. + case (i, value) => + if (currMax(i) < value) { + currMax(i) = value + } + if (currMin(i) > value) { + currMin(i) = value + } + + val tmpPrevMean = currMean(i) + currMean(i) = (currMean(i) * nnz(i) + value) / (nnz(i) + 1.0) + currM2n(i) += (value - currMean(i)) * (value - tmpPrevMean) + + nnz(i) += 1.0 + } + + totalCnt += 1.0 + this + } + + /** + * Merges another aggregator. + */ + def merge(other: ColumnStatisticsAggregator): this.type = { + require(n == other.n, s"Dimensions mismatch. Expecting $n but got ${other.n}.") + + totalCnt += other.totalCnt + val deltaMean = currMean - other.currMean + + var i = 0 + while (i < n) { + // merge mean together + if (other.currMean(i) != 0.0) { + currMean(i) = (currMean(i) * nnz(i) + other.currMean(i) * other.nnz(i)) / + (nnz(i) + other.nnz(i)) + } + // merge m2n together + if (nnz(i) + other.nnz(i) != 0.0) { + currM2n(i) += other.currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * other.nnz(i) / + (nnz(i) + other.nnz(i)) + } + if (currMax(i) < other.currMax(i)) { + currMax(i) = other.currMax(i) + } + if (currMin(i) > other.currMin(i)) { + currMin(i) = other.currMin(i) + } + i += 1 + } + + nnz += other.nnz + this + } +} /** * :: Experimental :: @@ -182,13 +314,7 @@ class RowMatrix( combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => (s1._1 + s2._1, s1._2 += s2._2) ) - // Update _m if it is not set, or verify its value. - if (nRows <= 0L) { - nRows = m - } else { - require(nRows == m, - s"The number of rows $m is different from what specified or previously computed: ${nRows}.") - } + updateNumRows(m) mean :/= m.toDouble @@ -240,6 +366,19 @@ class RowMatrix( } } + /** + * Computes column-wise summary statistics. + */ + def computeColumnSummaryStatistics(): MultivariateStatisticalSummary = { + val zeroValue = new ColumnStatisticsAggregator(numCols().toInt) + val summary = rows.map(_.toBreeze).aggregate[ColumnStatisticsAggregator](zeroValue)( + (aggregator, data) => aggregator.add(data), + (aggregator1, aggregator2) => aggregator1.merge(aggregator2) + ) + updateNumRows(summary.count) + summary + } + /** * Multiply this matrix by a local matrix on the right. * @@ -276,6 +415,16 @@ class RowMatrix( } mat } + + /** Updates or verfires the number of rows. */ + private def updateNumRows(m: Long) { + if (nRows <= 0) { + nRows == m + } else { + require(nRows == m, + s"The number of rows $m is different from what specified or previously computed: ${nRows}.") + } + } } object RowMatrix { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala new file mode 100644 index 0000000000000..f9eb343da2b82 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +/** + * Trait for multivariate statistical summary of a data matrix. + */ +trait MultivariateStatisticalSummary { + + /** + * Sample mean vector. + */ + def mean: Vector + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + def variance: Vector + + /** + * Sample size. + */ + def count: Long + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + def numNonzeros: Vector + + /** + * Maximum value of each column. + */ + def max: Vector + + /** + * Minimum value of each column. + */ + def min: Vector +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index ac2360c429e2b..901c3180eac4c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -17,14 +17,13 @@ package org.apache.spark.mllib.util -import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, - squaredDistance => breezeSquaredDistance} +import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.Vectors /** * Helper methods to load, save and pre-process data used in ML Lib. @@ -158,58 +157,6 @@ 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 numFeatures - number of features - * @param numExamples - 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. - */ - private[mllib] def computeStats( - data: RDD[LabeledPoint], - numFeatures: Int, - numExamples: Long): (Double, Vector, Vector) = { - val brzData = data.map { case LabeledPoint(label, features) => - (label, features.toBreeze) - } - val aggStats = brzData.aggregate( - (0L, 0.0, BDV.zeros[Double](numFeatures), BDV.zeros[Double](numFeatures)) - )( - seqOp = (c, v) => (c, v) match { - case ((n, sumLabel, sum, sumSq), (label, features)) => - features.activeIterator.foreach { case (i, x) => - sumSq(i) += x * x - } - (n + 1L, sumLabel + label, sum += features, sumSq) - }, - combOp = (c1, c2) => (c1, c2) match { - case ((n1, sumLabel1, sum1, sumSq1), (n2, sumLabel2, sum2, sumSq2)) => - (n1 + n2, sumLabel1 + sumLabel2, sum1 += sum2, sumSq1 += sumSq2) - } - ) - val (nl, sumLabel, sum, sumSq) = aggStats - - require(nl > 0, "Input data is empty.") - require(nl == numExamples) - - val n = nl.toDouble - val yMean = sumLabel / n - val mean = sum / n - val std = new Array[Double](sum.length) - var i = 0 - while (i < numFeatures) { - std(i) = sumSq(i) / n - mean(i) * mean(i) - i += 1 - } - - (yMean, Vectors.fromBreeze(mean), Vectors.dense(std)) - } - /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 71ee8e8a4f6fd..c9f9acf4c1335 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -170,4 +170,19 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { )) } } + + test("compute column summary statistics") { + for (mat <- Seq(denseMat, sparseMat)) { + val summary = mat.computeColumnSummaryStatistics() + // Run twice to make sure no internal states are changed. + for (k <- 0 to 1) { + assert(summary.mean === Vectors.dense(4.5, 3.0, 4.0), "mean mismatch") + assert(summary.variance === Vectors.dense(15.0, 10.0, 10.0), "variance mismatch") + assert(summary.count === m, "count mismatch.") + assert(summary.numNonzeros === Vectors.dense(3.0, 3.0, 4.0), "nnz mismatch") + assert(summary.max === Vectors.dense(9.0, 7.0, 8.0), "max mismatch") + assert(summary.min === Vectors.dense(0.0, 0.0, 1.0), "column mismatch.") + } + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index e451c350b8d88..812a8434784be 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -27,7 +27,6 @@ import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ class MLUtilsSuite extends FunSuite with LocalSparkContext { @@ -56,18 +55,6 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } - test("compute stats") { - val data = Seq.fill(3)(Seq( - LabeledPoint(1.0, Vectors.dense(1.0, 2.0, 3.0)), - LabeledPoint(0.0, Vectors.dense(3.0, 4.0, 5.0)) - )).flatten - val rdd = sc.parallelize(data, 2) - val (meanLabel, mean, std) = MLUtils.computeStats(rdd, 3, 6) - assert(meanLabel === 0.5) - assert(mean === Vectors.dense(2.0, 3.0, 4.0)) - assert(std === Vectors.dense(1.0, 1.0, 1.0)) - } - test("loadLibSVMData") { val lines = """ From aa8bb117a3ff98420ab751ba4ddbaad88ab57f9d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Fri, 11 Apr 2014 20:33:42 -0700 Subject: [PATCH 19/61] Update WindowedDStream.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit update the content of Exception when windowDuration is not multiple of parent.slideDuration Author: baishuo(白硕) Closes #390 from baishuo/windowdstream and squashes the following commits: 533c968 [baishuo(白硕)] Update WindowedDStream.scala --- .../org/apache/spark/streaming/dstream/WindowedDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index 24289b714f99e..775b6bfd065c0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -32,7 +32,7 @@ class WindowedDStream[T: ClassTag]( extends DStream[T](parent.ssc) { if (!_windowDuration.isMultipleOf(parent.slideDuration)) { - throw new Exception("The window duration of windowed DStream (" + _slideDuration + ") " + + throw new Exception("The window duration of windowed DStream (" + _windowDuration + ") " + "must be a multiple of the slide duration of parent DStream (" + parent.slideDuration + ")") } From 165e06a74c3d75e6b7341c120943add8b035b96a Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 11 Apr 2014 22:46:47 -0700 Subject: [PATCH 20/61] SPARK-1057 (alternative) Remove fastutil (This is for discussion at this point -- I'm not suggesting this should be committed.) This is what removing fastutil looks like. Much of it is straightforward, like using `java.io` buffered stream classes, and Guava for murmurhash3. Uses of the `FastByteArrayOutputStream` were a little trickier. In only one case though do I think the change to use `java.io` actually entails an extra array copy. The rest is using `OpenHashMap` and `OpenHashSet`. These are now written in terms of more scala-like operations. `OpenHashMap` is where I made three non-trivial changes to make it work, and they need review: - It is no longer private - The key must be a `ClassTag` - Unless a lot of other code changes, the key type can't enforce being a supertype of `Null` It all works and tests pass, and I think there is reason to believe it's OK from a speed perspective. But what about those last changes? Author: Sean Owen Closes #266 from srowen/SPARK-1057-alternate and squashes the following commits: 2601129 [Sean Owen] Fix Map return type error not previously caught ec65502 [Sean Owen] Updates from matei's review 00bc81e [Sean Owen] Remove use of fastutil and replace with use of java.io, spark.util and Guava classes --- core/pom.xml | 4 --- .../spark/broadcast/HttpBroadcast.scala | 9 +++-- .../spark/partial/GroupedCountEvaluator.scala | 32 +++++++---------- .../main/scala/org/apache/spark/rdd/RDD.scala | 34 +++++++++---------- .../spark/scheduler/ReplayListenerBus.scala | 5 ++- .../org/apache/spark/scheduler/Task.scala | 9 ++--- .../apache/spark/serializer/Serializer.scala | 9 ++--- .../apache/spark/storage/BlockManager.scala | 10 +++--- .../spark/storage/BlockObjectWriter.scala | 6 ++-- .../org/apache/spark/util/FileLogger.scala | 5 ++- .../org/apache/spark/util/SizeEstimator.scala | 5 ++- .../spark/util/collection/AppendOnlyMap.scala | 7 ++-- .../collection/ExternalAppendOnlyMap.scala | 5 ++- .../spark/util/collection/OpenHashMap.scala | 3 +- .../spark/util/collection/OpenHashSet.scala | 4 +-- pom.xml | 5 --- project/SparkBuild.scala | 1 - .../spark/streaming/util/RawTextHelper.scala | 15 ++++---- .../spark/streaming/util/RawTextSender.scala | 11 +++--- 19 files changed, 72 insertions(+), 107 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 1f808380817c9..a1bdd8ec68aeb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -157,10 +157,6 @@ - - it.unimi.dsi - fastutil - colt colt diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index f6a8a8af91e4b..29372f16f2cac 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -18,11 +18,10 @@ package org.apache.spark.broadcast import java.io.{File, FileOutputStream, ObjectInputStream, ObjectOutputStream, OutputStream} -import java.net.{URI, URL, URLConnection} +import java.io.{BufferedInputStream, BufferedOutputStream} +import java.net.{URL, URLConnection, URI} import java.util.concurrent.TimeUnit -import it.unimi.dsi.fastutil.io.{FastBufferedInputStream, FastBufferedOutputStream} - import org.apache.spark.{HttpServer, Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} @@ -164,7 +163,7 @@ private[spark] object HttpBroadcast extends Logging { if (compress) { compressionCodec.compressedOutputStream(new FileOutputStream(file)) } else { - new FastBufferedOutputStream(new FileOutputStream(file), bufferSize) + new BufferedOutputStream(new FileOutputStream(file), bufferSize) } } val ser = SparkEnv.get.serializer.newInstance() @@ -195,7 +194,7 @@ private[spark] object HttpBroadcast extends Logging { if (compress) { compressionCodec.compressedInputStream(inputStream) } else { - new FastBufferedInputStream(inputStream, bufferSize) + new BufferedInputStream(inputStream, bufferSize) } } val ser = SparkEnv.get.serializer.newInstance() diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index 40b70baabcad9..8bb78123e3c9c 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -22,36 +22,33 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.Map import scala.collection.mutable.HashMap +import scala.reflect.ClassTag import cern.jet.stat.Probability -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} + +import org.apache.spark.util.collection.OpenHashMap /** * An ApproximateEvaluator for counts by key. Returns a map of key to confidence interval. */ -private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Double) - extends ApproximateEvaluator[OLMap[T], Map[T, BoundedDouble]] { +private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, confidence: Double) + extends ApproximateEvaluator[OpenHashMap[T,Long], Map[T, BoundedDouble]] { var outputsMerged = 0 - var sums = new OLMap[T] // Sum of counts for each key + var sums = new OpenHashMap[T,Long]() // Sum of counts for each key - override def merge(outputId: Int, taskResult: OLMap[T]) { + override def merge(outputId: Int, taskResult: OpenHashMap[T,Long]) { outputsMerged += 1 - val iter = taskResult.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - sums.put(entry.getKey, sums.getLong(entry.getKey) + entry.getLongValue) + taskResult.foreach { case (key, value) => + sums.changeValue(key, value, _ + value) } } override def currentResult(): Map[T, BoundedDouble] = { if (outputsMerged == totalOutputs) { val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getLongValue() - result(entry.getKey) = new BoundedDouble(sum, 1.0, sum, sum) + sums.foreach { case (key, sum) => + result(key) = new BoundedDouble(sum, 1.0, sum, sum) } result } else if (outputsMerged == 0) { @@ -60,16 +57,13 @@ private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Dou val p = outputsMerged.toDouble / totalOutputs val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2) val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getLongValue + sums.foreach { case (key, sum) => val mean = (sum + 1 - p) / p val variance = (sum + 1) * (1 - p) / (p * p) val stdev = math.sqrt(variance) val low = mean - confFactor * stdev val high = mean + confFactor * stdev - result(entry.getKey) = new BoundedDouble(mean, confidence, low, high) + result(key) = new BoundedDouble(mean, confidence, low, high) } result } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3437b2cac19c2..891efccf23b6a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -20,12 +20,10 @@ package org.apache.spark.rdd import java.util.Random import scala.collection.Map -import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} import com.clearspring.analytics.stream.cardinality.HyperLogLog -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable @@ -43,6 +41,7 @@ import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel import org.apache.spark.util.{BoundedPriorityQueue, SerializableHyperLogLog, Utils} +import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler} /** @@ -834,24 +833,24 @@ abstract class RDD[T: ClassTag]( throw new SparkException("countByValue() does not support arrays") } // TODO: This should perhaps be distributed by default. - def countPartition(iter: Iterator[T]): Iterator[OLMap[T]] = { - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) + def countPartition(iter: Iterator[T]): Iterator[OpenHashMap[T,Long]] = { + val map = new OpenHashMap[T,Long] + iter.foreach { + t => map.changeValue(t, 1L, _ + 1L) } Iterator(map) } - def mergeMaps(m1: OLMap[T], m2: OLMap[T]): OLMap[T] = { - val iter = m2.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - m1.put(entry.getKey, m1.getLong(entry.getKey) + entry.getLongValue) + def mergeMaps(m1: OpenHashMap[T,Long], m2: OpenHashMap[T,Long]): OpenHashMap[T,Long] = { + m2.foreach { case (key, value) => + m1.changeValue(key, value, _ + value) } m1 } val myResult = mapPartitions(countPartition).reduce(mergeMaps) - myResult.asInstanceOf[java.util.Map[T, Long]] // Will be wrapped as a Scala mutable Map + // Convert to a Scala mutable map + val mutableResult = scala.collection.mutable.Map[T,Long]() + myResult.foreach { case (k, v) => mutableResult.put(k, v) } + mutableResult } /** @@ -866,11 +865,10 @@ abstract class RDD[T: ClassTag]( if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } - val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) + val countPartition: (TaskContext, Iterator[T]) => OpenHashMap[T,Long] = { (ctx, iter) => + val map = new OpenHashMap[T,Long] + iter.foreach { + t => map.changeValue(t, 1L, _ + 1L) } map } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index b03665fd56d33..f868e772cf58a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -17,11 +17,10 @@ package org.apache.spark.scheduler -import java.io.InputStream +import java.io.{BufferedInputStream, InputStream} import scala.io.Source -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ @@ -62,7 +61,7 @@ private[spark] class ReplayListenerBus( var currentLine = "" try { fileStream = Some(fileSystem.open(path)) - bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) + bufferedStream = Some(new BufferedInputStream(fileStream.get)) compressStream = Some(wrapForCompression(bufferedStream.get)) // Parse each line as an event and post the event to all attached listeners diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index b85b4a50cd93a..a8bcb7dfe2f3c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -17,13 +17,11 @@ package org.apache.spark.scheduler -import java.io.{DataInputStream, DataOutputStream} +import java.io.{ByteArrayOutputStream, DataInputStream, DataOutputStream} import java.nio.ByteBuffer import scala.collection.mutable.HashMap -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.TaskContext import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance @@ -104,7 +102,7 @@ private[spark] object Task { serializer: SerializerInstance) : ByteBuffer = { - val out = new FastByteArrayOutputStream(4096) + val out = new ByteArrayOutputStream(4096) val dataOut = new DataOutputStream(out) // Write currentFiles @@ -125,8 +123,7 @@ private[spark] object Task { dataOut.flush() val taskBytes = serializer.serialize(task).array() out.write(taskBytes) - out.trim() - ByteBuffer.wrap(out.array) + ByteBuffer.wrap(out.toByteArray) } /** diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 9f04dc6e427c0..f2c8f9b6218d6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -17,11 +17,9 @@ package org.apache.spark.serializer -import java.io.{EOFException, InputStream, OutputStream} +import java.io.{ByteArrayOutputStream, EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.SparkEnv import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.{ByteBufferInputStream, NextIterator} @@ -73,10 +71,9 @@ trait SerializerInstance { def serializeMany[T](iterator: Iterator[T]): ByteBuffer = { // Default implementation uses serializeStream - val stream = new FastByteArrayOutputStream() + val stream = new ByteArrayOutputStream() serializeStream(stream).writeAll(iterator) - val buffer = ByteBuffer.allocate(stream.position.toInt) - buffer.put(stream.array, 0, stream.position.toInt) + val buffer = ByteBuffer.wrap(stream.toByteArray) buffer.flip() buffer } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index df9bb4044e37a..f14017051fa07 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{File, InputStream, OutputStream} +import java.io.{File, InputStream, OutputStream, BufferedOutputStream, ByteArrayOutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -26,7 +26,6 @@ import scala.concurrent.duration._ import scala.util.Random import akka.actor.{ActorSystem, Cancellable, Props} -import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer import org.apache.spark.{Logging, MapOutputTracker, SecurityManager, SparkConf, SparkEnv, SparkException} @@ -992,7 +991,7 @@ private[spark] class BlockManager( outputStream: OutputStream, values: Iterator[Any], serializer: Serializer = defaultSerializer) { - val byteStream = new FastBufferedOutputStream(outputStream) + val byteStream = new BufferedOutputStream(outputStream) val ser = serializer.newInstance() ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() } @@ -1002,10 +1001,9 @@ private[spark] class BlockManager( blockId: BlockId, values: Iterator[Any], serializer: Serializer = defaultSerializer): ByteBuffer = { - val byteStream = new FastByteArrayOutputStream(4096) + val byteStream = new ByteArrayOutputStream(4096) dataSerializeStream(blockId, byteStream, values, serializer) - byteStream.trim() - ByteBuffer.wrap(byteStream.array) + ByteBuffer.wrap(byteStream.toByteArray) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 696b930a26b9e..a2687e6be4e34 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -17,11 +17,9 @@ package org.apache.spark.storage -import java.io.{FileOutputStream, File, OutputStream} +import java.io.{BufferedOutputStream, FileOutputStream, File, OutputStream} import java.nio.channels.FileChannel -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream - import org.apache.spark.Logging import org.apache.spark.serializer.{SerializationStream, Serializer} @@ -119,7 +117,7 @@ private[spark] class DiskBlockObjectWriter( ts = new TimeTrackingOutputStream(fos) channel = fos.getChannel() lastValidPosition = initialPosition - bs = compressStream(new FastBufferedOutputStream(ts, bufferSize)) + bs = compressStream(new BufferedOutputStream(ts, bufferSize)) objOut = serializer.newInstance().serializeStream(bs) initialized = true this diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 0080a8b342b05..68a12e8ed67d7 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -17,12 +17,11 @@ package org.apache.spark.util -import java.io._ +import java.io.{FileOutputStream, BufferedOutputStream, PrintWriter, IOException} import java.net.URI import java.text.SimpleDateFormat import java.util.Date -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import org.apache.hadoop.fs.{FSDataOutputStream, Path} import org.apache.spark.{Logging, SparkConf} @@ -100,7 +99,7 @@ private[spark] class FileLogger( hadoopDataStream.get } - val bstream = new FastBufferedOutputStream(dstream, outputBufferSize) + val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream new PrintWriter(cstream) } diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index b955612ca7749..08465575309c6 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -27,9 +27,8 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer -import it.unimi.dsi.fastutil.ints.IntOpenHashSet - import org.apache.spark.Logging +import org.apache.spark.util.collection.OpenHashSet /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in @@ -207,7 +206,7 @@ private[spark] object SizeEstimator extends Logging { // Estimate the size of a large array by sampling elements without replacement. var size = 0.0 val rand = new Random(42) - val drawn = new IntOpenHashSet(ARRAY_SAMPLE_SIZE) + val drawn = new OpenHashSet[Int](ARRAY_SAMPLE_SIZE) for (i <- 0 until ARRAY_SAMPLE_SIZE) { var index = 0 do { diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 025492b177a77..ad38250ad339f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -19,6 +19,8 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} +import com.google.common.hash.Hashing + import org.apache.spark.annotation.DeveloperApi /** @@ -199,11 +201,8 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) /** * Re-hash a value to deal better with hash functions that don't differ in the lower bits. - * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ - private def rehash(h: Int): Int = { - it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) - } + private def rehash(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() /** Double the table's size and re-hash everything */ protected def growTable() { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index dd01ae821f705..d615767284c0b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -17,14 +17,13 @@ package org.apache.spark.util.collection -import java.io._ +import java.io.{InputStream, BufferedInputStream, FileInputStream, File, Serializable, EOFException} import java.util.Comparator import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import com.google.common.io.ByteStreams -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.annotation.DeveloperApi @@ -350,7 +349,7 @@ class ExternalAppendOnlyMap[K, V, C]( private class DiskMapIterator(file: File, blockId: BlockId, batchSizes: ArrayBuffer[Long]) extends Iterator[(K, C)] { private val fileStream = new FileInputStream(file) - private val bufferedStream = new FastBufferedInputStream(fileStream, fileBufferSize) + private val bufferedStream = new BufferedInputStream(fileStream, fileBufferSize) // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index 62f99f3981793..b8de4ff9aa494 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -30,7 +30,8 @@ import org.apache.spark.annotation.DeveloperApi * Under the hood, it uses our OpenHashSet implementation. */ @DeveloperApi -class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( +private[spark] +class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] with Serializable { diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 148c12e64d2ce..19af4f8cbe428 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -18,6 +18,7 @@ package org.apache.spark.util.collection import scala.reflect._ +import com.google.common.hash.Hashing /** * A simple, fast hash set optimized for non-null insertion-only use case, where keys are never @@ -256,9 +257,8 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( /** * Re-hash a value to deal better with hash functions that don't differ in the lower bits. - * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ - private def hashcode(h: Int): Int = it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) + private def hashcode(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() private def nextPowerOf2(n: Int): Int = { val highBit = Integer.highestOneBit(n) diff --git a/pom.xml b/pom.xml index c03bb35c99442..5f66cbe768592 100644 --- a/pom.xml +++ b/pom.xml @@ -348,11 +348,6 @@ - - it.unimi.dsi - fastutil - 6.4.4 - colt colt diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 21163760e6277..a6058bba3d211 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -331,7 +331,6 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-slf4j" % akkaVersion excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), - "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", "commons-net" % "commons-net" % "2.2", diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index bd1df55cf70f5..bbf57ef9275c0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -19,18 +19,17 @@ package org.apache.spark.streaming.util import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} +import org.apache.spark.util.collection.OpenHashMap import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { - /** - * Splits lines and counts the words in them using specialized object-to-long hashmap - * (to avoid boxing-unboxing overhead of Long in java/scala HashMap) + /** + * Splits lines and counts the words. */ def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = { - val map = new OLMap[String] + val map = new OpenHashMap[String,Long] var i = 0 var j = 0 while (iter.hasNext) { @@ -43,14 +42,16 @@ object RawTextHelper { } if (j > i) { val w = s.substring(i, j) - val c = map.getLong(w) - map.put(w, c + 1) + map.changeValue(w, 1L, _ + 1L) } i = j while (i < s.length && s.charAt(i) == ' ') { i += 1 } } + map.toIterator.map { + case (k, v) => (k, v) + } } map.toIterator.map{case (k, v) => (k, v)} } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index 684b38e8b3102..a7850812bd612 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -17,14 +17,12 @@ package org.apache.spark.streaming.util -import java.io.IOException +import java.io.{ByteArrayOutputStream, IOException} import java.net.ServerSocket import java.nio.ByteBuffer import scala.io.Source -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.{SparkConf, Logging} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.IntParam @@ -45,16 +43,15 @@ object RawTextSender extends Logging { // Repeat the input data multiple times to fill in a buffer val lines = Source.fromFile(file).getLines().toArray - val bufferStream = new FastByteArrayOutputStream(blockSize + 1000) + val bufferStream = new ByteArrayOutputStream(blockSize + 1000) val ser = new KryoSerializer(new SparkConf()).newInstance() val serStream = ser.serializeStream(bufferStream) var i = 0 - while (bufferStream.position < blockSize) { + while (bufferStream.size < blockSize) { serStream.writeObject(lines(i)) i = (i + 1) % lines.length } - bufferStream.trim() - val array = bufferStream.array + val array = bufferStream.toByteArray val countBuf = ByteBuffer.wrap(new Array[Byte](4)) countBuf.putInt(array.length) From 6aa08c39cf30fa5c4ed97f4fff16371b9030a2e6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 11 Apr 2014 23:33:49 -0700 Subject: [PATCH 21/61] [SPARK-1386] Web UI for Spark Streaming When debugging Spark Streaming applications it is necessary to monitor certain metrics that are not shown in the Spark application UI. For example, what is average processing time of batches? What is the scheduling delay? Is the system able to process as fast as it is receiving data? How many records I am receiving through my receivers? While the StreamingListener interface introduced in the 0.9 provided some of this information, it could only be accessed programmatically. A UI that shows information specific to the streaming applications is necessary for easier debugging. This PR introduces such a UI. It shows various statistics related to the streaming application. Here is a screenshot of the UI running on my local machine. http://i.imgur.com/1ooDGhm.png This UI is integrated into the Spark UI running at 4040. Author: Tathagata Das Author: Andrew Or Closes #290 from tdas/streaming-web-ui and squashes the following commits: fc73ca5 [Tathagata Das] Merge pull request #9 from andrewor14/ui-refactor 642dd88 [Andrew Or] Merge SparkUISuite.scala into UISuite.scala eb30517 [Andrew Or] Merge github.com:apache/spark into ui-refactor f4f4cbe [Tathagata Das] More minor fixes. 34bb364 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 252c566 [Tathagata Das] Merge pull request #8 from andrewor14/ui-refactor e038b4b [Tathagata Das] Addressed Patrick's comments. 125a054 [Andrew Or] Disable serving static resources with gzip 90feb8d [Andrew Or] Address Patrick's comments 89dae36 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 72fe256 [Tathagata Das] Merge pull request #6 from andrewor14/ui-refactor 2fc09c8 [Tathagata Das] Added binary check exclusions aa396d4 [Andrew Or] Rename tabs and pages (No more IndexPage.scala) f8e1053 [Tathagata Das] Added Spark and Streaming UI unit tests. caa5e05 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 585cd65 [Tathagata Das] Merge pull request #5 from andrewor14/ui-refactor 914b8ff [Tathagata Das] Moved utils functions to UIUtils. 548c98c [Andrew Or] Wide refactoring of WebUI, UITab, and UIPage (see commit message) 6de06b0 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui ee6543f [Tathagata Das] Minor changes based on Andrew's comments. fa760fe [Tathagata Das] Fixed long line. 1c0bcef [Tathagata Das] Refactored streaming UI into two files. 1af239b [Tathagata Das] Changed streaming UI to attach itself as a tab with the Spark UI. 827e81a [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 168fe86 [Tathagata Das] Merge pull request #2 from andrewor14/ui-refactor 3e986f8 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui c78c92d [Andrew Or] Remove outdated comment 8f7323b [Andrew Or] End of file new lines, indentation, and imports (minor) 0d61ee8 [Andrew Or] Merge branch 'streaming-web-ui' of github.com:tdas/spark into ui-refactor 9a48fa1 [Andrew Or] Allow adding tabs to SparkUI dynamically + add example 61358e3 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-web-ui 53be2c5 [Tathagata Das] Minor style updates. ed25dfc [Andrew Or] Generalize SparkUI header to display tabs dynamically a37ad4f [Andrew Or] Comments, imports and formatting (minor) cd000b0 [Andrew Or] Merge github.com:apache/spark into ui-refactor 7d57444 [Andrew Or] Refactoring the UI interface to add flexibility aef4dd5 [Tathagata Das] Added Apache licenses. db27bad [Tathagata Das] Added last batch processing time to StreamingUI. 4d86e98 [Tathagata Das] Added basic stats to the StreamingUI and refactored the UI to a Page to make it easier to transition to using SparkUI later. 93f1c69 [Tathagata Das] Added network receiver information to the Streaming UI. 56cc7fb [Tathagata Das] First cut implementation of Streaming UI. --- .../scala/org/apache/spark/SparkContext.scala | 1 - .../spark/deploy/SparkUIContainer.scala | 50 ----- .../{IndexPage.scala => HistoryPage.scala} | 12 +- .../spark/deploy/history/HistoryServer.scala | 61 +++--- .../apache/spark/deploy/master/Master.scala | 8 +- .../deploy/master/ui/ApplicationPage.scala | 13 +- .../ui/{IndexPage.scala => MasterPage.scala} | 23 ++- .../spark/deploy/master/ui/MasterWebUI.scala | 54 ++---- .../apache/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/ui/LogPage.scala | 147 ++++++++++++++ .../ui/{IndexPage.scala => WorkerPage.scala} | 6 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 180 +++--------------- .../scheduler/ApplicationEventListener.scala | 4 +- .../apache/spark/storage/StorageUtils.scala | 16 +- .../org/apache/spark/ui/JettyUtils.scala | 1 + .../main/scala/org/apache/spark/ui/Page.scala | 22 --- .../scala/org/apache/spark/ui/SparkUI.scala | 108 ++++------- .../scala/org/apache/spark/ui/UIUtils.scala | 172 +++++++++++++---- .../scala/org/apache/spark/ui/WebUI.scala | 141 +++++++++++--- ...ironmentUI.scala => EnvironmentPage.scala} | 47 +---- .../apache/spark/ui/env/EnvironmentTab.scala | 50 +++++ ...{ExecutorsUI.scala => ExecutorsPage.scala} | 84 +------- .../apache/spark/ui/exec/ExecutorsTab.scala | 86 +++++++++ .../apache/spark/ui/jobs/ExecutorTable.scala | 7 +- .../spark/ui/jobs/JobProgressListener.scala | 10 +- ...{IndexPage.scala => JobProgressPage.scala} | 16 +- ...bProgressUI.scala => JobProgressTab.scala} | 45 ++--- .../org/apache/spark/ui/jobs/PoolPage.scala | 14 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 7 +- .../org/apache/spark/ui/jobs/StagePage.scala | 45 ++--- .../org/apache/spark/ui/jobs/StageTable.scala | 18 +- .../org/apache/spark/ui/storage/RDDPage.scala | 17 +- .../{IndexPage.scala => StoragePage.scala} | 13 +- ...{BlockManagerUI.scala => StorageTab.scala} | 32 +--- .../org/apache/spark/util/JsonProtocol.scala | 12 +- .../scala/org/apache/spark/ui/UISuite.scala | 81 +++++++- .../apache/spark/util/JsonProtocolSuite.scala | 4 +- project/MimaBuild.scala | 8 +- .../spark/streaming/StreamingContext.scala | 23 +-- .../spark/streaming/dstream/DStream.scala | 9 - .../dstream/NetworkInputDStream.scala | 79 +++++--- .../spark/streaming/scheduler/BatchInfo.scala | 1 + .../streaming/scheduler/JobGenerator.scala | 9 +- .../streaming/scheduler/JobScheduler.scala | 11 +- .../spark/streaming/scheduler/JobSet.scala | 7 +- .../scheduler/NetworkInputTracker.scala | 86 ++++++--- .../scheduler/StreamingListener.scala | 18 +- .../scheduler/StreamingListenerBus.scala | 4 + .../ui/StreamingJobProgressListener.scala | 148 ++++++++++++++ .../spark/streaming/ui/StreamingPage.scala | 180 ++++++++++++++++++ .../spark/streaming/ui/StreamingTab.scala | 27 +-- .../spark/streaming/InputStreamsSuite.scala | 6 +- .../streaming/StreamingContextSuite.scala | 1 - .../org/apache/spark/streaming/UISuite.scala | 46 +++++ 54 files changed, 1426 insertions(+), 846 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala rename core/src/main/scala/org/apache/spark/deploy/history/{IndexPage.scala => HistoryPage.scala} (85%) rename core/src/main/scala/org/apache/spark/deploy/master/ui/{IndexPage.scala => MasterPage.scala} (91%) create mode 100644 core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala rename core/src/main/scala/org/apache/spark/deploy/worker/ui/{IndexPage.scala => WorkerPage.scala} (97%) delete mode 100644 core/src/main/scala/org/apache/spark/ui/Page.scala rename core/src/main/scala/org/apache/spark/ui/env/{EnvironmentUI.scala => EnvironmentPage.scala} (61%) create mode 100644 core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala rename core/src/main/scala/org/apache/spark/ui/exec/{ExecutorsUI.scala => ExecutorsPage.scala} (61%) create mode 100644 core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala rename core/src/main/scala/org/apache/spark/ui/jobs/{IndexPage.scala => JobProgressPage.scala} (90%) rename core/src/main/scala/org/apache/spark/ui/jobs/{JobProgressUI.scala => JobProgressTab.scala} (53%) rename core/src/main/scala/org/apache/spark/ui/storage/{IndexPage.scala => StoragePage.scala} (90%) rename core/src/main/scala/org/apache/spark/ui/storage/{BlockManagerUI.scala => StorageTab.scala} (75%) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala rename core/src/test/scala/org/apache/spark/SparkUISuite.scala => streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala (58%) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3bcc8ce2b25a6..a764c174d562c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -213,7 +213,6 @@ class SparkContext(config: SparkConf) extends Logging { // Initialize the Spark UI, registering all associated listeners private[spark] val ui = new SparkUI(this) ui.bind() - ui.start() // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala deleted file mode 100644 index 33fceae4ff489..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy - -import org.apache.spark.ui.{SparkUI, WebUI} - -private[spark] abstract class SparkUIContainer(name: String) extends WebUI(name) { - - /** Attach a SparkUI to this container. Only valid after bind(). */ - def attachUI(ui: SparkUI) { - assert(serverInfo.isDefined, - "%s must be bound to a server before attaching SparkUIs".format(name)) - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - rootHandler.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } - } - } - - /** Detach a SparkUI from this container. Only valid after bind(). */ - def detachUI(ui: SparkUI) { - assert(serverInfo.isDefined, - "%s must be bound to a server before detaching SparkUIs".format(name)) - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - if (handler.isStarted) { - handler.stop() - } - rootHandler.removeHandler(handler) - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala similarity index 85% rename from core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 54dffffec71c5..180c853ce3096 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -21,9 +21,9 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{UIUtils, WebUI} +import org.apache.spark.ui.{WebUIPage, UIUtils} -private[spark] class IndexPage(parent: HistoryServer) { +private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } @@ -62,13 +62,13 @@ private[spark] class IndexPage(parent: HistoryServer) { private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { val appName = if (info.started) info.name else info.logDirPath.getName val uiAddress = parent.getAddress + info.ui.basePath - val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" - val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not completed" + val startTime = if (info.started) UIUtils.formatDate(info.startTime) else "Not started" + val endTime = if (info.completed) UIUtils.formatDate(info.endTime) else "Not completed" val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L - val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" + val duration = if (difference > 0) UIUtils.formatDuration(difference) else "---" val sparkUser = if (info.started) info.sparkUser else "Unknown user" val logDirectory = info.logDirPath.getName - val lastUpdated = WebUI.formatDate(info.lastUpdated) + val lastUpdated = UIUtils.formatDate(info.lastUpdated) {appName} {startTime} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 97d2ba9deed33..cf64700f9098c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -17,17 +17,13 @@ package org.apache.spark.deploy.history -import javax.servlet.http.HttpServletRequest - import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} -import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.scheduler._ -import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.{WebUI, SparkUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils @@ -46,17 +42,15 @@ import org.apache.spark.util.Utils */ class HistoryServer( val baseLogDir: String, + securityManager: SecurityManager, conf: SparkConf) - extends SparkUIContainer("History Server") with Logging { + extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging { import HistoryServer._ private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) - private val port = WEB_UI_PORT - private val securityManager = new SecurityManager(conf) - private val indexPage = new IndexPage(this) // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTime = -1L @@ -90,37 +84,23 @@ class HistoryServer( } } - private val handlers = Seq[ServletContextHandler]( - createStaticHandler(STATIC_RESOURCE_DIR, "/static"), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager) - ) - // A mapping of application ID to its history information, which includes the rendered UI val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + initialize() + /** - * Start the history server. + * Initialize the history server. * * This starts a background thread that periodically synchronizes information displayed on * this UI with the event logs in the provided base directory. */ - def start() { + def initialize() { + attachPage(new HistoryPage(this)) + attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) logCheckingThread.start() } - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) - logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to bind HistoryServer", e) - System.exit(1) - } - } - /** * Check for any updates to event logs in the base directory. This is only effective once * the server has been bound. @@ -151,7 +131,7 @@ class HistoryServer( // Remove any applications that should no longer be retained appIdToInfo.foreach { case (appId, info) => if (!retainedAppIds.contains(appId)) { - detachUI(info.ui) + detachSparkUI(info.ui) appIdToInfo.remove(appId) } } @@ -186,15 +166,14 @@ class HistoryServer( val path = logDir.getPath val appId = path.getName val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) - val ui = new SparkUI(replayBus, appId, "/history/" + appId) val appListener = new ApplicationEventListener replayBus.addListener(appListener) + val ui = new SparkUI(conf, replayBus, appId, "/history/" + appId) // Do not call ui.bind() to avoid creating a new server for each application - ui.start() replayBus.replay() if (appListener.applicationStarted) { - attachUI(ui) + attachSparkUI(ui) val appName = appListener.appName val sparkUser = appListener.sparkUser val startTime = appListener.startTime @@ -213,6 +192,18 @@ class HistoryServer( fileSystem.close() } + /** Attach a reconstructed UI to this server. Only valid after bind(). */ + private def attachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") + ui.getHandlers.foreach(attachHandler) + } + + /** Detach a reconstructed UI from this server. Only valid after bind(). */ + private def detachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") + ui.getHandlers.foreach(detachHandler) + } + /** Return the address of this server. */ def getAddress: String = "http://" + publicHost + ":" + boundPort @@ -262,9 +253,9 @@ object HistoryServer { def main(argStrings: Array[String]) { val args = new HistoryServerArguments(argStrings) - val server = new HistoryServer(args.logDir, conf) + val securityManager = new SecurityManager(conf) + val server = new HistoryServer(args.logDir, securityManager, conf) server.bind() - server.start() // Wait until the end of the world... or if the HistoryServer process is manually stopped while(true) { Thread.sleep(Int.MaxValue) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2446e86cb6672..6c58e741df001 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -625,7 +625,7 @@ private[spark] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach( a => { - appIdToUI.remove(a.id).foreach { ui => webUi.detachUI(ui) } + appIdToUI.remove(a.id).foreach { ui => webUi.detachSparkUI(ui) } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) @@ -667,12 +667,12 @@ private[spark] class Master( if (!eventLogPaths.isEmpty) { try { val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) - val ui = new SparkUI(replayBus, appName + " (completed)", "/history/" + app.id) - ui.start() + val ui = new SparkUI( + new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id) replayBus.replay() app.desc.appUiUrl = ui.basePath appIdToUI(app.id) = ui - webUi.attachUI(ui) + webUi.attachSparkUI(ui) return true } catch { case t: Throwable => diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index cb092cb5d576b..b5cd4d2ea963f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -28,15 +28,16 @@ import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class ApplicationPage(parent: MasterWebUI) { - val master = parent.masterActorRef - val timeout = parent.timeout +private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { + + private val master = parent.masterActorRef + private val timeout = parent.timeout /** Executor details for a particular application */ - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) @@ -96,7 +97,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { UIUtils.basicSparkPage(content, "Application: " + app.desc.name) } - def executorRow(executor: ExecutorInfo): Seq[Node] = { + private def executorRow(executor: ExecutorInfo): Seq[Node] = { {executor.id} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala similarity index 91% rename from core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 8c1d6c7cce450..7ca3b08a28728 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -25,17 +25,17 @@ import scala.xml.Node import akka.pattern.ask import org.json4s.JValue -import org.apache.spark.deploy.{JsonProtocol} +import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: MasterWebUI) { - val master = parent.masterActorRef - val timeout = parent.timeout +private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { + private val master = parent.masterActorRef + private val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) JsonProtocol.writeMasterState(state) @@ -139,7 +139,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { UIUtils.basicSparkPage(content, "Spark Master at " + state.uri) } - def workerRow(worker: WorkerInfo): Seq[Node] = { + private def workerRow(worker: WorkerInfo): Seq[Node] = { {worker.id} @@ -154,8 +154,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { } - - def appRow(app: ApplicationInfo): Seq[Node] = { + private def appRow(app: ApplicationInfo): Seq[Node] = { {app.id} @@ -169,14 +168,14 @@ private[spark] class IndexPage(parent: MasterWebUI) { {Utils.megabytesToString(app.desc.memoryPerSlave)} - {WebUI.formatDate(app.submitDate)} + {UIUtils.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} - {WebUI.formatDuration(app.duration)} + {UIUtils.formatDuration(app.duration)} } - def driverRow(driver: DriverInfo): Seq[Node] = { + private def driverRow(driver: DriverInfo): Seq[Node] = { {driver.id} {driver.submitDate} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 30c8ade408a5a..a18b39fc95d64 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,14 +17,9 @@ package org.apache.spark.deploy.master.ui -import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.Logging -import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -33,44 +28,33 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) - extends SparkUIContainer("MasterWebUI") with Logging { + extends WebUI(master.securityMgr, requestedPort, master.conf) with Logging { val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) - private val host = Utils.localHostName() - private val port = requestedPort - private val applicationPage = new ApplicationPage(this) - private val indexPage = new IndexPage(this) + initialize() - private val handlers: Seq[ServletContextHandler] = { - master.masterMetricsSystem.getServletHandlers ++ - master.applicationMetricsSystem.getServletHandlers ++ - Seq[ServletContextHandler]( - createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"), - createServletHandler("/app/json", - (request: HttpServletRequest) => applicationPage.renderJson(request), master.securityMgr), - createServletHandler("/app", - (request: HttpServletRequest) => applicationPage.render(request), master.securityMgr), - createServletHandler("/json", - (request: HttpServletRequest) => indexPage.renderJson(request), master.securityMgr), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), master.securityMgr) - ) + /** Initialize all components of the server. */ + def initialize() { + attachPage(new ApplicationPage(this)) + attachPage(new MasterPage(this)) + attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) + master.masterMetricsSystem.getServletHandlers.foreach(attachHandler) + master.applicationMetricsSystem.getServletHandlers.foreach(attachHandler) } - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Master web UI", e) - System.exit(1) - } + /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ + def attachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") + ui.getHandlers.foreach(attachHandler) } + /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ + def detachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") + ui.getHandlers.foreach(detachHandler) + } } private[spark] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index bf5a8d09dd2df..52c164ca3c574 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -128,8 +128,8 @@ private[spark] class Worker( host, port, cores, Utils.megabytesToString(memory))) logInfo("Spark home: " + sparkHome) createWorkDir() - webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.bind() registerWithMaster() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala new file mode 100644 index 0000000000000..fec1207948628 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.worker.ui + +import java.io.File +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.util.Utils + +private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { + private val worker = parent.worker + private val workDir = parent.workDir + + def renderLog(request: HttpServletRequest): String = { + val defaultBytes = 100 * 1024 + + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + + val path = (appId, executorId, driverId) match { + case (Some(a), Some(e), None) => + s"${workDir.getPath}/$appId/$executorId/$logType" + case (None, None, Some(d)) => + s"${workDir.getPath}/$driverId/$logType" + case _ => + throw new Exception("Request must specify either application or driver identifiers") + } + + val (startByte, endByte) = getByteRange(path, offset, byteLength) + val file = new File(path) + val logLength = file.length + + val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n" + pre + Utils.offsetBytes(path, startByte, endByte) + } + + def render(request: HttpServletRequest): Seq[Node] = { + val defaultBytes = 100 * 1024 + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + + val (path, params) = (appId, executorId, driverId) match { + case (Some(a), Some(e), None) => + (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e") + case (None, None, Some(d)) => + (s"${workDir.getPath}/$d/$logType", s"driverId=$d") + case _ => + throw new Exception("Request must specify either application or driver identifiers") + } + + val (startByte, endByte) = getByteRange(path, offset, byteLength) + val file = new File(path) + val logLength = file.length + val logText = {Utils.offsetBytes(path, startByte, endByte)} + val linkToMaster =

    Back to Master

    + val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} + + val backButton = + if (startByte > 0) { + + + + } + else { + + } + + val nextButton = + if (endByte < logLength) { + + + + } + else { + + } + + val content = + + + {linkToMaster} +
    +
    {backButton}
    +
    {range}
    +
    {nextButton}
    +
    +
    +
    +
    {logText}
    +
    + + + UIUtils.basicSparkPage(content, logType + " log page for " + appId) + } + + /** Determine the byte range for a log or log page. */ + private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { + val defaultBytes = 100 * 1024 + val maxBytes = 1024 * 1024 + val file = new File(path) + val logLength = file.length() + val getOffset = offset.getOrElse(logLength - defaultBytes) + val startByte = + if (getOffset < 0) 0L + else if (getOffset > logLength) logLength + else getOffset + val logPageLength = math.min(byteLength, maxBytes) + val endByte = math.min(startByte + logPageLength, logLength) + (startByte, endByte) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 49c1009cac2bf..d4513118ced05 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -28,15 +28,15 @@ import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: WorkerWebUI) { +private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, timeout) JsonProtocol.writeWorkerState(workerState) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 5625a44549aaa..0ad2edba2227f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -20,174 +20,44 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} +import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends WebUI("WorkerWebUI") with Logging { +class WorkerWebUI( + val worker: Worker, + val workDir: File, + port: Option[Int] = None) + extends WebUI(worker.securityMgr, WorkerWebUI.getUIPort(port, worker.conf), worker.conf) + with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) - private val host = Utils.localHostName() - private val port = requestedPort.getOrElse( - worker.conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) - private val indexPage = new IndexPage(this) - - private val handlers: Seq[ServletContextHandler] = { - worker.metricsSystem.getServletHandlers ++ - Seq[ServletContextHandler]( - createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static"), - createServletHandler("/log", - (request: HttpServletRequest) => log(request), worker.securityMgr), - createServletHandler("/logPage", - (request: HttpServletRequest) => logPage(request), worker.securityMgr), - createServletHandler("/json", - (request: HttpServletRequest) => indexPage.renderJson(request), worker.securityMgr), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), worker.securityMgr) - ) - } - - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, worker.conf)) - logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Worker web UI", e) - System.exit(1) - } - } - - private def log(request: HttpServletRequest): String = { - val defaultBytes = 100 * 1024 - - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - - val path = (appId, executorId, driverId) match { - case (Some(a), Some(e), None) => - s"${workDir.getPath}/$appId/$executorId/$logType" - case (None, None, Some(d)) => - s"${workDir.getPath}/$driverId/$logType" - case _ => - throw new Exception("Request must specify either application or driver identifiers") - } - - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - - val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n" - pre + Utils.offsetBytes(path, startByte, endByte) - } - - private def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { - val defaultBytes = 100 * 1024 - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - - val (path, params) = (appId, executorId, driverId) match { - case (Some(a), Some(e), None) => - (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e") - case (None, None, Some(d)) => - (s"${workDir.getPath}/$d/$logType", s"driverId=$d") - case _ => - throw new Exception("Request must specify either application or driver identifiers") - } - - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - val logText = {Utils.offsetBytes(path, startByte, endByte)} - val linkToMaster =

    Back to Master

    - val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} - - val backButton = - if (startByte > 0) { - - - - } - else { - - } - - val nextButton = - if (endByte < logLength) { - - - - } - else { - - } - - val content = - - - {linkToMaster} -
    -
    {backButton}
    -
    {range}
    -
    {nextButton}
    -
    -
    -
    -
    {logText}
    -
    - - - UIUtils.basicSparkPage(content, logType + " log page for " + appId) + initialize() + + /** Initialize all components of the server. */ + def initialize() { + val logPage = new LogPage(this) + attachPage(logPage) + attachPage(new WorkerPage(this)) + attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static")) + attachHandler(createServletHandler("/log", + (request: HttpServletRequest) => logPage.renderLog(request), worker.securityMgr)) + worker.metricsSystem.getServletHandlers.foreach(attachHandler) } - - /** Determine the byte range for a log or log page. */ - private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { - val defaultBytes = 100 * 1024 - val maxBytes = 1024 * 1024 - val file = new File(path) - val logLength = file.length() - val getOffset = offset.getOrElse(logLength - defaultBytes) - val startByte = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset - val logPageLength = math.min(byteLength, maxBytes) - val endByte = math.min(startByte + logPageLength, logLength) - (startByte, endByte) - } - } private[spark] object WorkerWebUI { - val DEFAULT_PORT=8081 + val DEFAULT_PORT = 8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR + + def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = { + requestedPort.getOrElse(conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index affda13df6531..c1001227151a5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -31,11 +31,11 @@ private[spark] class ApplicationEventListener extends SparkListener { def applicationStarted = startTime != -1 - def applicationFinished = endTime != -1 + def applicationCompleted = endTime != -1 def applicationDuration: Long = { val difference = endTime - startTime - if (applicationStarted && applicationFinished && difference > 0) difference else -1L + if (applicationStarted && applicationCompleted && difference > 0) difference else -1L } override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 07255aa366a6d..7ed371326855d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -42,24 +42,22 @@ class StorageStatus( def memRemaining : Long = maxMem - memUsed() - def rddBlocks = blocks.flatMap { - case (rdd: RDDBlockId, status) => Some(rdd, status) - case _ => None - } + def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } } @DeveloperApi private[spark] class RDDInfo( - val id: Int, - val name: String, - val numPartitions: Int, - val storageLevel: StorageLevel) extends Ordered[RDDInfo] { + val id: Int, + val name: String, + val numPartitions: Int, + val storageLevel: StorageLevel) + extends Ordered[RDDInfo] { var numCachedPartitions = 0 var memSize = 0L var diskSize = 0L - var tachyonSize= 0L + var tachyonSize = 0L override def toString = { import Utils.bytesToString diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index dd0818e8ab01c..62a4e3d0f6a42 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -121,6 +121,7 @@ private[spark] object JettyUtils extends Logging { /** Create a handler for serving files from a static directory */ def createStaticHandler(resourceBase: String, path: String): ServletContextHandler = { val contextHandler = new ServletContextHandler + contextHandler.setInitParameter("org.eclipse.jetty.servlet.Default.gzip", "false") val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) Option(getClass.getClassLoader.getResource(resourceBase)) match { diff --git a/core/src/main/scala/org/apache/spark/ui/Page.scala b/core/src/main/scala/org/apache/spark/ui/Page.scala deleted file mode 100644 index b2a069a37552d..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/Page.scala +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ui - -private[spark] object Page extends Enumeration { - val Stages, Storage, Environment, Executors = Value -} diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 7fa4fd3149eb6..2fef1a635427c 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,112 +17,86 @@ package org.apache.spark.ui -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.env.EnvironmentUI -import org.apache.spark.ui.exec.ExecutorsUI -import org.apache.spark.ui.jobs.JobProgressUI -import org.apache.spark.ui.storage.BlockManagerUI -import org.apache.spark.util.Utils +import org.apache.spark.ui.env.EnvironmentTab +import org.apache.spark.ui.exec.ExecutorsTab +import org.apache.spark.ui.jobs.JobProgressTab +import org.apache.spark.ui.storage.StorageTab -/** Top level user interface for Spark */ +/** + * Top level user interface for a Spark application. + */ private[spark] class SparkUI( val sc: SparkContext, val conf: SparkConf, + val securityManager: SecurityManager, val listenerBus: SparkListenerBus, var appName: String, val basePath: String = "") - extends WebUI("SparkUI") with Logging { + extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath) + with Logging { - def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) - def this(listenerBus: SparkListenerBus, appName: String, basePath: String) = - this(null, new SparkConf, listenerBus, appName, basePath) + def this(sc: SparkContext) = this(sc, sc.conf, sc.env.securityManager, sc.listenerBus, sc.appName) + def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, conf, new SecurityManager(conf), listenerBus, appName, basePath) // If SparkContext is not provided, assume the associated application is not live val live = sc != null - val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - - private val localHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) - private val port = conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + // Maintain executor storage status through Spark events + val storageStatusListener = new StorageStatusListener - private val storage = new BlockManagerUI(this) - private val jobs = new JobProgressUI(this) - private val env = new EnvironmentUI(this) - private val exec = new ExecutorsUI(this) + initialize() - val handlers: Seq[ServletContextHandler] = { - val metricsServletHandlers = if (live) { - SparkEnv.get.metricsSystem.getServletHandlers - } else { - Array[ServletContextHandler]() + /** Initialize all components of the server. */ + def initialize() { + listenerBus.addListener(storageStatusListener) + val jobProgressTab = new JobProgressTab(this) + attachTab(jobProgressTab) + attachTab(new StorageTab(this)) + attachTab(new EnvironmentTab(this)) + attachTab(new ExecutorsTab(this)) + attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) + attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) + attachHandler( + createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) + if (live) { + sc.env.metricsSystem.getServletHandlers.foreach(attachHandler) } - storage.getHandlers ++ - jobs.getHandlers ++ - env.getHandlers ++ - exec.getHandlers ++ - metricsServletHandlers ++ - Seq[ServletContextHandler] ( - createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"), - createRedirectHandler("/", "/stages", basePath = basePath) - ) } - // Maintain executor storage status through Spark events - val storageStatusListener = new StorageStatusListener - + /** Set the app name for this UI. */ def setAppName(name: String) { appName = name } - /** Initialize all components of the server */ - def start() { - storage.start() - jobs.start() - env.start() - exec.start() - - // Storage status listener must receive events first, as other listeners depend on its state - listenerBus.addListener(storageStatusListener) - listenerBus.addListener(storage.listener) - listenerBus.addListener(jobs.listener) - listenerBus.addListener(env.listener) - listenerBus.addListener(exec.listener) - } - - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) - logInfo("Started Spark web UI at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Spark web UI", e) - System.exit(1) - } + /** Register the given listener with the listener bus. */ + def registerListener(listener: SparkListener) { + listenerBus.addListener(listener) } /** Stop the server behind this web interface. Only valid after bind(). */ override def stop() { super.stop() - logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) + logInfo("Stopped Spark web UI at %s".format(appUIAddress)) } /** * Return the application UI host:port. This does not include the scheme (http://). */ - private[spark] def appUIHostPort = publicHost + ":" + boundPort + private[spark] def appUIHostPort = publicHostName + ":" + boundPort private[spark] def appUIAddress = s"http://$appUIHostPort" - } private[spark] object SparkUI { val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + + def getUIPort(conf: SparkConf): Int = { + conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a7cf04b3cbb86..6a2d652528d8a 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -17,16 +17,115 @@ package org.apache.spark.ui +import java.text.SimpleDateFormat +import java.util.{Locale, Date} + import scala.xml.Node +import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ -private[spark] object UIUtils { +private[spark] object UIUtils extends Logging { + + // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. + private val dateFormat = new ThreadLocal[SimpleDateFormat]() { + override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + } + + def formatDate(date: Date): String = dateFormat.get.format(date) + + def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) + + def formatDuration(milliseconds: Long): String = { + val seconds = milliseconds.toDouble / 1000 + if (seconds < 60) { + return "%.0f s".format(seconds) + } + val minutes = seconds / 60 + if (minutes < 10) { + return "%.1f min".format(minutes) + } else if (minutes < 60) { + return "%.0f min".format(minutes) + } + val hours = minutes / 60 + "%.1f h".format(hours) + } + + /** Generate a verbose human-readable string representing a duration such as "5 second 35 ms" */ + def formatDurationVerbose(ms: Long): String = { + try { + val second = 1000L + val minute = 60 * second + val hour = 60 * minute + val day = 24 * hour + val week = 7 * day + val year = 365 * day + + def toString(num: Long, unit: String): String = { + if (num == 0) { + "" + } else if (num == 1) { + s"$num $unit" + } else { + s"$num ${unit}s" + } + } + + val millisecondsString = if (ms >= second && ms % second == 0) "" else s"${ms % second} ms" + val secondString = toString((ms % minute) / second, "second") + val minuteString = toString((ms % hour) / minute, "minute") + val hourString = toString((ms % day) / hour, "hour") + val dayString = toString((ms % week) / day, "day") + val weekString = toString((ms % year) / week, "week") + val yearString = toString(ms / year, "year") - import Page._ + Seq( + second -> millisecondsString, + minute -> s"$secondString $millisecondsString", + hour -> s"$minuteString $secondString", + day -> s"$hourString $minuteString $secondString", + week -> s"$dayString $hourString $minuteString", + year -> s"$weekString $dayString $hourString" + ).foreach { case (durationLimit, durationString) => + if (ms < durationLimit) { + // if time is less than the limit (upto year) + return durationString + } + } + // if time is more than a year + return s"$yearString $weekString $dayString" + } catch { + case e: Exception => + logError("Error converting time to string", e) + // if there is some error, return blank string + return "" + } + } + + /** Generate a human-readable string representing a number (e.g. 100 K) */ + def formatNumber(records: Double): String = { + val trillion = 1e12 + val billion = 1e9 + val million = 1e6 + val thousand = 1e3 + + val (value, unit) = { + if (records >= 2*trillion) { + (records / trillion, " T") + } else if (records >= 2*billion) { + (records / billion, " B") + } else if (records >= 2*million) { + (records / million, " M") + } else if (records >= 2*thousand) { + (records / thousand, " K") + } else { + (records, "") + } + } + "%.1f%s".formatLocal(Locale.US, value, unit) + } // Yarn has to go through a proxy so the base uri is provided and has to be on all links - private[spark] val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")). - getOrElse("") + val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).getOrElse("") def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource @@ -36,26 +135,14 @@ private[spark] object UIUtils { basePath: String, appName: String, title: String, - page: Page.Value) : Seq[Node] = { - val jobs = page match { - case Stages => -
  • Stages
  • - case _ =>
  • Stages
  • - } - val storage = page match { - case Storage => -
  • Storage
  • - case _ =>
  • Storage
  • - } - val environment = page match { - case Environment => -
  • Environment
  • - case _ =>
  • Environment
  • - } - val executors = page match { - case Executors => -
  • Executors
  • - case _ =>
  • Executors
  • + tabs: Seq[WebUITab], + activeTab: WebUITab, + refreshInterval: Option[Int] = None): Seq[Node] = { + + val header = tabs.map { tab => +
  • + {tab.name} +
  • } @@ -74,16 +161,10 @@ private[spark] object UIUtils { - + -
    @@ -129,21 +210,36 @@ private[spark] object UIUtils { /** 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], + generateDataRow: T => Seq[Node], + data: Seq[T], fixedWidth: Boolean = false): Seq[Node] = { - val colWidth = 100.toDouble / headers.size - val colWidthAttr = if (fixedWidth) colWidth + "%" else "" var tableClass = "table table-bordered table-striped table-condensed sortable" if (fixedWidth) { tableClass += " table-fixed" } - + val colWidth = 100.toDouble / headers.size + val colWidthAttr = if (fixedWidth) colWidth + "%" else "" + val headerRow: Seq[Node] = { + // if none of the headers have "\n" in them + if (headers.forall(!_.contains("\n"))) { + // represent header as simple text + headers.map(h => {h}) + } else { + // represent header text as list while respecting "\n" + headers.map { case h => + +
      + { h.split("\n").map { case t =>
    • {t}
    • } } +
    + + } + } + } - {headers.map(h => )} + {headerRow} - {rows.map(r => makeRow(r))} + {data.map(r => generateDataRow(r))}
    {h}
    } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 2cc7582eca8a3..b08f308fda1dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -17,53 +17,134 @@ package org.apache.spark.ui -import java.text.SimpleDateFormat -import java.util.Date +import javax.servlet.http.HttpServletRequest -private[spark] abstract class WebUI(name: String) { +import scala.collection.mutable.ArrayBuffer +import scala.xml.Node + +import org.eclipse.jetty.servlet.ServletContextHandler +import org.json4s.JsonAST.{JNothing, JValue} + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils + +/** + * The top level component of the UI hierarchy that contains the server. + * + * Each WebUI represents a collection of tabs, each of which in turn represents a collection of + * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly. + */ +private[spark] abstract class WebUI( + securityManager: SecurityManager, + port: Int, + conf: SparkConf, + basePath: String = "") + extends Logging { + + protected val tabs = ArrayBuffer[WebUITab]() + protected val handlers = ArrayBuffer[ServletContextHandler]() protected var serverInfo: Option[ServerInfo] = None + protected val localHostName = Utils.localHostName() + protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) + private val className = Utils.getFormattedClassName(this) + + def getTabs: Seq[WebUITab] = tabs.toSeq + def getHandlers: Seq[ServletContextHandler] = handlers.toSeq + + /** Attach a tab to this UI, along with all of its attached pages. */ + def attachTab(tab: WebUITab) { + tab.pages.foreach(attachPage) + tabs += tab + } + + /** Attach a page to this UI. */ + def attachPage(page: WebUIPage) { + val pagePath = "/" + page.prefix + attachHandler(createServletHandler(pagePath, + (request: HttpServletRequest) => page.render(request), securityManager, basePath)) + attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json", + (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)) + } + + /** Attach a handler to this UI. */ + def attachHandler(handler: ServletContextHandler) { + handlers += handler + serverInfo.foreach { info => + info.rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } - /** - * Bind to the HTTP server behind this web interface. - * Overridden implementation should set serverInfo. - */ - def bind() { } + /** Detach a handler from this UI. */ + def detachHandler(handler: ServletContextHandler) { + handlers -= handler + serverInfo.foreach { info => + info.rootHandler.removeHandler(handler) + if (handler.isStarted) { + handler.stop() + } + } + } + + /** Initialize all components of the server. */ + def initialize() + + /** Bind to the HTTP server behind this web interface. */ + def bind() { + assert(!serverInfo.isDefined, "Attempted to bind %s more than once!".format(className)) + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + logInfo("Started %s at http://%s:%d".format(className, publicHostName, boundPort)) + } catch { + case e: Exception => + logError("Failed to bind %s".format(className), e) + System.exit(1) + } + } /** Return the actual port to which this server is bound. Only valid after bind(). */ def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) /** Stop the server behind this web interface. Only valid after bind(). */ def stop() { - assert(serverInfo.isDefined, "Attempted to stop %s before binding to a server!".format(name)) + assert(serverInfo.isDefined, + "Attempted to stop %s before binding to a server!".format(className)) serverInfo.get.server.stop() } } + /** - * Utilities used throughout the web UI. + * A tab that represents a collection of pages. + * The prefix is appended to the parent address to form a full path, and must not contain slashes. */ -private[spark] object WebUI { - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. - private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") +private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { + val pages = ArrayBuffer[WebUIPage]() + val name = prefix.capitalize + + /** Attach a page to this tab. This prepends the page's prefix with the tab's own prefix. */ + def attachPage(page: WebUIPage) { + page.prefix = (prefix + "/" + page.prefix).stripSuffix("/") + pages += page } - def formatDate(date: Date): String = dateFormat.get.format(date) + /** Get a list of header tabs from the parent UI. */ + def headerTabs: Seq[WebUITab] = parent.getTabs +} - def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) - def formatDuration(milliseconds: Long): String = { - val seconds = milliseconds.toDouble / 1000 - if (seconds < 60) { - return "%.0f s".format(seconds) - } - val minutes = seconds / 60 - if (minutes < 10) { - return "%.1f min".format(minutes) - } else if (minutes < 60) { - return "%.0f min".format(minutes) - } - val hours = minutes / 60 - "%.1f h".format(hours) - } +/** + * A page that represents the leaf node in the UI hierarchy. + * + * The direct parent of a WebUIPage is not specified as it can be either a WebUI or a WebUITab. + * If the parent is a WebUI, the prefix is appended to the parent's address to form a full path. + * Else, if the parent is a WebUITab, the prefix is appended to the super prefix of the parent + * to form a relative path. The prefix must not contain slashes. + */ +private[spark] abstract class WebUIPage(var prefix: String) { + def render(request: HttpServletRequest): Seq[Node] + def renderJson(request: HttpServletRequest): JValue = JNothing } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala rename to core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index 33df97187ea78..b347eb1b83c1f 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -21,29 +21,12 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.eclipse.jetty.servlet.ServletContextHandler +import org.apache.spark.ui.{UIUtils, WebUIPage} -import org.apache.spark.scheduler._ -import org.apache.spark.ui._ -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.Page.Environment - -private[ui] class EnvironmentUI(parent: SparkUI) { +private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath - private var _listener: Option[EnvironmentListener] = None - - private def appName = parent.appName - - lazy val listener = _listener.get - - def start() { - _listener = Some(new EnvironmentListener) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/environment", - (request: HttpServletRequest) => render(request), parent.securityManager, basePath) - ) + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val runtimeInformationTable = UIUtils.listingTable( @@ -62,7 +45,7 @@ private[ui] class EnvironmentUI(parent: SparkUI) {

    Classpath Entries

    {classpathEntriesTable} - UIUtils.headerSparkPage(content, basePath, appName, "Environment", Environment) + UIUtils.headerSparkPage(content, basePath, appName, "Environment", parent.headerTabs, parent) } private def propertyHeader = Seq("Name", "Value") @@ -71,23 +54,3 @@ private[ui] class EnvironmentUI(parent: SparkUI) { private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} private def classPathRow(data: (String, String)) = {data._1}{data._2} } - -/** - * A SparkListener that prepares information to be displayed on the EnvironmentUI - */ -private[ui] class EnvironmentListener extends SparkListener { - var jvmInformation = Seq[(String, String)]() - var sparkProperties = Seq[(String, String)]() - var systemProperties = Seq[(String, String)]() - var classpathEntries = Seq[(String, String)]() - - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - synchronized { - val environmentDetails = environmentUpdate.environmentDetails - jvmInformation = environmentDetails("JVM Information") - sparkProperties = environmentDetails("Spark Properties") - systemProperties = environmentDetails("System Properties") - classpathEntries = environmentDetails("Classpath Entries") - } - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala new file mode 100644 index 0000000000000..03b46e1bd59af --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.env + +import org.apache.spark.scheduler._ +import org.apache.spark.ui._ + +private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "environment") { + val appName = parent.appName + val basePath = parent.basePath + val listener = new EnvironmentListener + + attachPage(new EnvironmentPage(this)) + parent.registerListener(listener) +} + +/** + * A SparkListener that prepares information to be displayed on the EnvironmentTab + */ +private[ui] class EnvironmentListener extends SparkListener { + var jvmInformation = Seq[(String, String)]() + var sparkProperties = Seq[(String, String)]() + var systemProperties = Seq[(String, String)]() + var classpathEntries = Seq[(String, String)]() + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + jvmInformation = environmentDetails("JVM Information") + sparkProperties = environmentDetails("Spark Properties") + systemProperties = environmentDetails("System Properties") + classpathEntries = environmentDetails("Classpath Entries") + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala rename to core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 77a38a1d3aa7c..c1e69f6cdaffb 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -19,35 +19,15 @@ package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.HashMap import scala.xml.Node -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.ExceptionFailure -import org.apache.spark.scheduler._ -import org.apache.spark.storage.StorageStatusListener -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui.{SparkUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[ui] class ExecutorsUI(parent: SparkUI) { +private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath - private var _listener: Option[ExecutorsListener] = None - - private def appName = parent.appName - - lazy val listener = _listener.get - - def start() { - _listener = Some(new ExecutorsListener(parent.storageStatusListener)) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/executors", - (request: HttpServletRequest) => render(request), parent.securityManager, basePath) - ) + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList @@ -75,8 +55,8 @@ private[ui] class ExecutorsUI(parent: SparkUI) {
    ; - UIUtils.headerSparkPage( - content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage(content, basePath, appName, "Executors (" + execInfo.size + ")", + parent.headerTabs, parent) } /** Header fields for the executors table */ @@ -159,55 +139,3 @@ private[ui] class ExecutorsUI(parent: SparkUI) { execFields.zip(execValues).toMap } } - -/** - * A SparkListener that prepares information to be displayed on the ExecutorsUI - */ -private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) - extends SparkListener { - - val executorToTasksActive = HashMap[String, Int]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - - def storageStatusList = storageStatusListener.storageStatusList - - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val eid = formatExecutorId(taskStart.taskInfo.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - val info = taskEnd.taskInfo - if (info != null) { - val eid = formatExecutorId(info.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 - executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - } - - // Update shuffle read/write - val metrics = taskEnd.taskMetrics - if (metrics != null) { - metrics.shuffleReadMetrics.foreach { shuffleRead => - executorToShuffleRead(eid) = - executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead - } - metrics.shuffleWriteMetrics.foreach { shuffleWrite => - executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten - } - } - } - } - - // This addresses executor ID inconsistencies in the local mode - private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) -} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala new file mode 100644 index 0000000000000..5678bf34ac730 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.exec + +import scala.collection.mutable.HashMap + +import org.apache.spark.ExceptionFailure +import org.apache.spark.scheduler._ +import org.apache.spark.storage.StorageStatusListener +import org.apache.spark.ui.{SparkUI, WebUITab} + +private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "executors") { + val appName = parent.appName + val basePath = parent.basePath + val listener = new ExecutorsListener(parent.storageStatusListener) + + attachPage(new ExecutorsPage(this)) + parent.registerListener(listener) +} + +/** + * A SparkListener that prepares information to be displayed on the ExecutorsTab + */ +private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) + extends SparkListener { + + val executorToTasksActive = HashMap[String, Int]() + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToDuration = HashMap[String, Long]() + val executorToShuffleRead = HashMap[String, Long]() + val executorToShuffleWrite = HashMap[String, Long]() + + def storageStatusList = storageStatusListener.storageStatusList + + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + val eid = formatExecutorId(taskStart.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val info = taskEnd.taskInfo + if (info != null) { + val eid = formatExecutorId(info.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 + executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + } + + // Update shuffle read/write + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { shuffleRead => + executorToShuffleRead(eid) = + executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead + } + metrics.shuffleWriteMetrics.foreach { shuffleWrite => + executorToShuffleWrite(eid) = + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + } + } + } + } + + // This addresses executor ID inconsistencies in the local mode + private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 73861ae6746da..c83e196c9c156 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -20,11 +20,12 @@ package org.apache.spark.ui.jobs import scala.collection.mutable import scala.xml.Node +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing executor summary */ -private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { - private lazy val listener = parent.listener +private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -69,7 +70,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { {k} {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} - {parent.formatDuration(v.taskTime)} + {UIUtils.formatDuration(v.taskTime)} {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 5167e20ea3d7d..0db4afa701b41 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -222,12 +222,10 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { - val schedulingModeName = - environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") - schedulingMode = schedulingModeName match { - case Some(name) => Some(SchedulingMode.withName(name)) - case None => None - } + schedulingMode = environmentUpdate + .environmentDetails("Spark Properties").toMap + .get("spark.scheduler.mode") + .map(SchedulingMode.withName) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala similarity index 90% rename from core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 8619a31380f1e..34ff2ac34a7ca 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -22,25 +22,23 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class IndexPage(parent: JobProgressUI) { +private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener + private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler - private def appName = parent.appName - def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - val now = System.currentTimeMillis() + val now = System.currentTimeMillis val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) @@ -59,7 +57,7 @@ private[ui] class IndexPage(parent: JobProgressUI) { // Total duration is not meaningful unless the UI is live
  • Total Duration: - {parent.formatDuration(now - sc.startTime)} + {UIUtils.formatDuration(now - sc.startTime)}
  • }}
  • @@ -94,7 +92,7 @@ private[ui] class IndexPage(parent: JobProgressUI) {

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", parent.headerTabs, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala similarity index 53% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index 30e3f35f2182b..3308c8c8a3d37 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -19,39 +19,28 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.SparkUI -import org.apache.spark.util.Utils +import org.apache.spark.ui.{SparkUI, WebUITab} /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressUI(parent: SparkUI) { +private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stages") { + val appName = parent.appName val basePath = parent.basePath val live = parent.live val sc = parent.sc - val killEnabled = parent.conf.getBoolean("spark.ui.killEnabled", true) - - lazy val listener = _listener.get - lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - - private val indexPage = new IndexPage(this) - private val stagePage = new StagePage(this) - private val poolPage = new PoolPage(this) - private var _listener: Option[JobProgressListener] = None + val conf = if (live) sc.conf else new SparkConf + val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) + val listener = new JobProgressListener(conf) - def appName = parent.appName + attachPage(new JobProgressPage(this)) + attachPage(new StagePage(this)) + attachPage(new PoolPage(this)) + parent.registerListener(listener) - def start() { - val conf = if (live) sc.conf else new SparkConf - _listener = Some(new JobProgressListener(conf)) - } - - def formatDuration(ms: Long) = Utils.msDurationToString(ms) + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - private def handleKillRequest(request: HttpServletRequest) = { + def handleKillRequest(request: HttpServletRequest) = { if (killEnabled) { val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt @@ -64,14 +53,4 @@ private[ui] class JobProgressUI(parent: SparkUI) { Thread.sleep(100) } } - - def getHandlers = Seq[ServletContextHandler]( - createRedirectHandler("/stages/stage/kill", "/stages", handleKillRequest), - createServletHandler("/stages/stage", - (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), - createServletHandler("/stages/pool", - (request: HttpServletRequest) => poolPage.render(request), parent.securityManager, basePath), - createServletHandler("/stages", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 3638e6035ba81..fd83d37583967 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -22,17 +22,15 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressUI) { +private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { + private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -52,8 +50,8 @@ private[ui] class PoolPage(parent: JobProgressUI) {

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - UIUtils.headerSparkPage( - content, basePath, appName, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Fair Scheduler Pool: " + poolName, + parent.headerTabs, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index c5c8d8668740b..f4b68f241966d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -24,10 +24,9 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { private val basePath = parent.basePath - private val poolToActiveStages = listener.poolToActiveStages - private lazy val listener = parent.listener + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -48,7 +47,7 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { SchedulingMode - {rows.map(r => makeRow(r, poolToActiveStages))} + {rows.map(r => makeRow(r, listener.poolToActiveStages))} } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index b6c3e3cf45163..4bce472036f7d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,17 +22,14 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressUI) { +private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { + private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener - private lazy val sc = parent.sc - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -44,8 +41,8 @@ private[ui] class StagePage(parent: JobProgressUI) {

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet
  • - return UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %s".format(stageId), Stages) + return UIUtils.headerSparkPage(content, basePath, appName, + "Details for Stage %s".format(stageId), parent.headerTabs, parent) } val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) @@ -60,7 +57,7 @@ private[ui] class StagePage(parent: JobProgressUI) { val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 var activeTime = 0L - val now = System.currentTimeMillis() + val now = System.currentTimeMillis val tasksActive = listener.stageIdToTasksActive(stageId).values tasksActive.foreach(activeTime += _.timeRunning(now)) @@ -70,7 +67,7 @@ private[ui] class StagePage(parent: JobProgressUI) {
    • Total task time across all tasks: - {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)} + {UIUtils.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
    • {if (hasShuffleRead)
    • @@ -121,13 +118,13 @@ private[ui] class StagePage(parent: JobProgressUI) { } val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes). - get.getQuantiles().map(ms => parent.formatDuration(ms.toLong)) + get.getQuantiles().map(ms => UIUtils.formatDuration(ms.toLong)) val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.executorRunTime.toDouble } val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles() - .map(ms => parent.formatDuration(ms.toLong)) + .map(ms => UIUtils.formatDuration(ms.toLong)) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { @@ -138,7 +135,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val gettingResultQuantiles = "Time spent fetching task results" +: Distribution(gettingResultTimes).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, @@ -155,7 +152,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val schedulerDelayQuantiles = "Scheduler delay" +: Distribution(schedulerDelays).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } def getQuantileCols(data: Seq[Double]) = @@ -206,8 +203,8 @@ private[ui] class StagePage(parent: JobProgressUI) {

      Aggregated Metrics by Executor

      ++ executorTable.toNodeSeq ++

      Tasks

      ++ taskTable - UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId), + parent.headerTabs, parent) } } @@ -219,8 +216,8 @@ private[ui] class StagePage(parent: JobProgressUI) { taskData match { case TaskUIData(info, metrics, exception) => val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) else metrics.map(_.executorRunTime).getOrElse(1L) - val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) - else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) + else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) @@ -235,8 +232,8 @@ private[ui] class StagePage(parent: JobProgressUI) { val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") - val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => - if (ms == 0) "" else parent.formatDuration(ms) + val writeTimeReadable = maybeWriteTime.map(t => t / (1000 * 1000)).map { ms => + if (ms == 0) "" else UIUtils.formatDuration(ms) }.getOrElse("") val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) @@ -254,15 +251,15 @@ private[ui] class StagePage(parent: JobProgressUI) { {info.status} {info.taskLocality} {info.host} - {WebUI.formatDate(new Date(info.launchTime))} + {UIUtils.formatDate(new Date(info.launchTime))} {formatDuration} - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} + {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} - {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} + {if (serializationTime > 0) UIUtils.formatDuration(serializationTime) else ""} {if (shuffleRead) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index e419fae5a6589..8c5b1f55fd2dc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -23,17 +23,17 @@ import scala.collection.mutable.HashMap import scala.xml.Node import org.apache.spark.scheduler.{StageInfo, TaskInfo} -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTable( - stages: Seq[StageInfo], - parent: JobProgressUI, - killEnabled: Boolean = false) { + stages: Seq[StageInfo], + parent: JobProgressTab, + killEnabled: Boolean = false) { private val basePath = parent.basePath - private lazy val listener = parent.listener + private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler def toNodeSeq: Seq[Node] = { @@ -89,25 +89,23 @@ private[ui] class StageTable( {s.name} - val description = listener.stageIdToDescription.get(s.stageId) + listener.stageIdToDescription.get(s.stageId) .map(d =>
      {d}
      {nameLink} {killLink}
      ) .getOrElse(
      {killLink}{nameLink}
      ) - - return description } /** Render an HTML row that represents a stage */ private def stageRow(s: StageInfo): Seq[Node] = { val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { - case Some(t) => WebUI.formatDate(new Date(t)) + case Some(t) => UIUtils.formatDate(new Date(t)) case None => "Unknown" } val finishTime = s.completionTime.getOrElse(System.currentTimeMillis) val duration = s.submissionTime.map { t => if (finishTime > t) finishTime - t else System.currentTimeMillis - t } - val formattedDuration = duration.map(d => parent.formatDuration(d)).getOrElse("Unknown") + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val startedTasks = listener.stageIdToTasksActive.getOrElse(s.stageId, HashMap[Long, TaskInfo]()).size val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 75ee9976d7b5f..d07f1c9b20fcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -22,23 +22,22 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RDDPage(parent: BlockManagerUI) { +private[ui] class RddPage(parent: StorageTab) extends WebUIPage("rdd") { + private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt val storageStatusList = listener.storageStatusList val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { // Rather than crashing, render an "RDD Not Found" page - return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", Storage) + return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", + parent.headerTabs, parent) } // Worker table @@ -96,8 +95,8 @@ private[ui] class RDDPage(parent: BlockManagerUI) { ; - UIUtils.headerSparkPage( - content, basePath, appName, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage(content, basePath, appName, "RDD Storage Info for " + rddInfo.name, + parent.headerTabs, parent) } /** Header fields for the worker table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala similarity index 90% rename from core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 4f6acc30a88c4..b66edd91f56c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -22,22 +22,19 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.RDDInfo -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[ui] class IndexPage(parent: BlockManagerUI) { +private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) - UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) + UIUtils.headerSparkPage(content, basePath, appName, "Storage ", parent.headerTabs, parent) } /** Header fields for the RDD table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala similarity index 75% rename from core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 16996a2da1e72..56429f6c07fcd 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -17,45 +17,27 @@ package org.apache.spark.ui.storage -import javax.servlet.http.HttpServletRequest - import scala.collection.mutable -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.ui._ -import org.apache.spark.ui.JettyUtils._ import org.apache.spark.scheduler._ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class BlockManagerUI(parent: SparkUI) { +private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage") { + val appName = parent.appName val basePath = parent.basePath + val listener = new StorageListener(parent.storageStatusListener) - private val indexPage = new IndexPage(this) - private val rddPage = new RDDPage(this) - private var _listener: Option[BlockManagerListener] = None - - lazy val listener = _listener.get - - def appName = parent.appName - - def start() { - _listener = Some(new BlockManagerListener(parent.storageStatusListener)) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/storage/rdd", - (request: HttpServletRequest) => rddPage.render(request), parent.securityManager, basePath), - createServletHandler("/storage", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) + attachPage(new StoragePage(this)) + attachPage(new RddPage(this)) + parent.registerListener(listener) } /** * A SparkListener that prepares information to be displayed on the BlockManagerUI */ -private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListener) +private[ui] class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private val _rddInfoMap = mutable.Map[Int, RDDInfo]() diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index f2396f7c80a35..465835ea7fe29 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -88,30 +88,27 @@ private[spark] object JsonProtocol { def taskStartToJson(taskStart: SparkListenerTaskStart): JValue = { val taskInfo = taskStart.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskStart)) ~ ("Stage ID" -> taskStart.stageId) ~ - ("Task Info" -> taskInfoJson) + ("Task Info" -> taskInfoToJson(taskInfo)) } def taskGettingResultToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { val taskInfo = taskGettingResult.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ - ("Task Info" -> taskInfoJson) + ("Task Info" -> taskInfoToJson(taskInfo)) } def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { val taskEndReason = taskEndReasonToJson(taskEnd.reason) val taskInfo = taskEnd.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing val taskMetrics = taskEnd.taskMetrics val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ ("Stage ID" -> taskEnd.stageId) ~ ("Task Type" -> taskEnd.taskType) ~ ("Task End Reason" -> taskEndReason) ~ - ("Task Info" -> taskInfoJson) ~ + ("Task Info" -> taskInfoToJson(taskInfo)) ~ ("Task Metrics" -> taskMetricsJson) } @@ -505,6 +502,9 @@ private[spark] object JsonProtocol { } def taskMetricsFromJson(json: JValue): TaskMetrics = { + if (json == JNothing) { + return TaskMetrics.empty + } val metrics = new TaskMetrics metrics.hostname = (json \ "Host Name").extract[String] metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 2f9739f940dc6..b85c483ca2a08 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -18,16 +18,81 @@ package org.apache.spark.ui import java.net.ServerSocket +import javax.servlet.http.HttpServletRequest +import scala.io.Source import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler import org.scalatest.FunSuite +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkConf +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.LocalSparkContext._ +import scala.xml.Node class UISuite extends FunSuite { + + test("basic ui visibility") { + withSpark(new SparkContext("local", "test")) { sc => + // test if the ui is visible, and all the expected tabs are visible + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + } + } + + test("visibility at localhost:4040") { + withSpark(new SparkContext("local", "test")) { sc => + // test if visible from http://localhost:4040 + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL("http://localhost:4040").mkString + assert(html.toLowerCase.contains("stages")) + } + } + } + + test("attaching a new tab") { + withSpark(new SparkContext("local", "test")) { sc => + val sparkUI = sc.ui + + val newTab = new WebUITab(sparkUI, "foo") { + attachPage(new WebUIPage("") { + def render(request: HttpServletRequest): Seq[Node] = { + "html magic" + } + }) + } + sparkUI.attachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + + // check whether new page exists + assert(html.toLowerCase.contains("foo")) + + // check whether other pages still exist + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress.stripSuffix("/") + "/foo").mkString + // check whether new page exists + assert(html.contains("magic")) + } + } + } + test("jetty port increases under contention") { val startPort = 4040 val server = new Server(startPort) @@ -60,4 +125,18 @@ class UISuite extends FunSuite { case Failure(e) => } } + + test("verify appUIAddress contains the scheme") { + withSpark(new SparkContext("local", "test")) { sc => + val uiAddress = sc.ui.appUIAddress + assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + } + } + + test("verify appUIAddress contains the port") { + withSpark(new SparkContext("local", "test")) { sc => + val splitUIAddress = sc.ui.appUIAddress.split(':') + assert(splitUIAddress(2).toInt == sc.ui.boundPort) + } + } } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index f75297a02dc8b..16470bb7bf60d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -523,8 +523,8 @@ class JsonProtocolSuite extends FunSuite { 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": - {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, - "Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false, + "Deserialized":false,"Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} """ private val jobStartJsonString = diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 5ea4817bfde18..9cb31d70444ff 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -60,6 +60,7 @@ object MimaBuild { Seq( excludePackage("org.apache.spark.api.java"), excludePackage("org.apache.spark.streaming.api.java"), + excludePackage("org.apache.spark.streaming.scheduler"), excludePackage("org.apache.spark.mllib") ) ++ excludeSparkClass("rdd.ClassTags") ++ @@ -70,7 +71,12 @@ object MimaBuild { excludeSparkClass("mllib.regression.LassoWithSGD") ++ excludeSparkClass("mllib.regression.LinearRegressionWithSGD") ++ excludeSparkClass("streaming.dstream.NetworkReceiver") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") + excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator#Block") ++ + excludeSparkClass("streaming.dstream.ReportError") ++ + excludeSparkClass("streaming.dstream.ReportBlock") ++ + excludeSparkClass("streaming.dstream.DStream") case _ => Seq() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index a4e236c65ff86..ff5d0aaa3d0bd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -17,29 +17,28 @@ package org.apache.spark.streaming -import scala.collection.mutable.Queue -import scala.collection.Map -import scala.reflect.ClassTag - import java.io.InputStream import java.util.concurrent.atomic.AtomicInteger -import akka.actor.Props -import akka.actor.SupervisorStrategy -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.Text +import scala.collection.Map +import scala.collection.mutable.Queue +import scala.reflect.ClassTag + +import akka.actor.{Props, SupervisorStrategy} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat -import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.MetadataCleaner import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receivers._ import org.apache.spark.streaming.scheduler._ -import org.apache.hadoop.conf.Configuration +import org.apache.spark.streaming.ui.StreamingTab +import org.apache.spark.util.MetadataCleaner /** * Main entry point for Spark Streaming functionality. It provides methods used to create @@ -158,6 +157,8 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter + private[streaming] val uiTab = new StreamingTab(this) + /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { type CheckpointState = Value diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d043200f71a0b..a7e5215437e54 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -353,15 +353,6 @@ abstract class DStream[T: ClassTag] ( dependencies.foreach(_.clearMetadata(time)) } - /* Adds metadata to the Stream while it is running. - * This method should be overwritten by sublcasses of InputDStream. - */ - private[streaming] def addMetadata(metadata: Any) { - if (metadata != null) { - logInfo("Dropping Metadata: " + metadata.toString) - } - } - /** * Refresh the list of checkpointed RDDs that will be saved along with checkpoint of * this stream. This is an internal method that should not be called directly. This is diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index d19a635fe8eca..5a249706b4d2f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -17,24 +17,23 @@ package org.apache.spark.streaming.dstream -import java.util.concurrent.{TimeUnit, ArrayBlockingQueue} import java.nio.ByteBuffer +import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.Await -import scala.concurrent.duration._ import scala.reflect.ClassTag -import akka.actor.{Props, Actor} +import akka.actor.{Actor, Props} import akka.pattern.ask -import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} -import org.apache.spark.streaming._ import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.rdd.{RDD, BlockRDD} +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} -import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver} -import org.apache.spark.util.AkkaUtils +import org.apache.spark.streaming._ +import org.apache.spark.streaming.scheduler.{AddBlock, DeregisterReceiver, ReceivedBlockInfo, RegisterReceiver} +import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} +import org.apache.spark.util.{AkkaUtils, Utils} /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -49,8 +48,10 @@ import org.apache.spark.util.AkkaUtils abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) extends InputDStream[T](ssc_) { - // This is an unique identifier that is used to match the network receiver with the - // corresponding network input stream. + /** Keeps all received blocks information */ + private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] + + /** This is an unique identifier for the network input stream. */ val id = ssc.getNewNetworkStreamId() /** @@ -65,25 +66,44 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte def stop() {} + /** Ask NetworkInputTracker for received data blocks and generates RDDs with them. */ override def compute(validTime: Time): Option[RDD[T]] = { // If this is called for any time before the start time of the context, // then this returns an empty RDD. This may happen when recovering from a // master failure if (validTime >= graph.startTime) { - val blockIds = ssc.scheduler.networkInputTracker.getBlocks(id, validTime) + val blockInfo = ssc.scheduler.networkInputTracker.getReceivedBlockInfo(id) + receivedBlockInfo(validTime) = blockInfo + val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) Some(new BlockRDD[T](ssc.sc, blockIds)) } else { Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) } } + + /** Get information on received blocks. */ + private[streaming] def getReceivedBlockInfo(time: Time) = { + receivedBlockInfo(time) + } + + /** + * Clear metadata that are older than `rememberDuration` of this DStream. + * This is an internal method that should not be called directly. This + * implementation overrides the default implementation to clear received + * block information. + */ + private[streaming] override def clearMetadata(time: Time) { + super.clearMetadata(time) + val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) + receivedBlockInfo --= oldReceivedBlocks.keys + logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + + (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) + } } private[streaming] sealed trait NetworkReceiverMessage -private[streaming] case class StopReceiver() extends NetworkReceiverMessage -private[streaming] case class ReportBlock(blockId: BlockId, metadata: Any) - extends NetworkReceiverMessage -private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage +private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage /** * Abstract class of a receiver that can be run on worker nodes to receive external data. See @@ -177,6 +197,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case (e, i) => "Exception " + i + ": " + e.getMessage + "\n" + e.getStackTraceString }.mkString("\n") } + logInfo("Deregistering receiver " + streamId) val future = trackerActor.ask(DeregisterReceiver(streamId, message))(askTimeout) Await.result(future, askTimeout) @@ -209,18 +230,28 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** * Push a block (as an ArrayBuffer filled with data) into the block manager. */ - def pushBlock(blockId: BlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { + def pushBlock( + blockId: StreamBlockId, + arrayBuffer: ArrayBuffer[T], + metadata: Any, + level: StorageLevel + ) { env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) - trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, arrayBuffer.size, metadata)) logDebug("Pushed block " + blockId) } /** * Push a block (as bytes) into the block manager. */ - def pushBlock(blockId: BlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) { + def pushBlock( + blockId: StreamBlockId, + bytes: ByteBuffer, + metadata: Any, + level: StorageLevel + ) { env.blockManager.putBytes(blockId, bytes, level) - trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, -1, metadata)) } /** Set the ID of the DStream that this receiver is associated with */ @@ -232,9 +263,11 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging private class NetworkReceiverActor extends Actor { override def preStart() { - logInfo("Registered receiver " + streamId) - val future = trackerActor.ask(RegisterReceiver(streamId, self))(askTimeout) + val msg = RegisterReceiver( + streamId, NetworkReceiver.this.getClass.getSimpleName, Utils.localHostName(), self) + val future = trackerActor.ask(msg)(askTimeout) Await.result(future, askTimeout) + logInfo("Registered receiver " + streamId) } override def receive() = { @@ -253,7 +286,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging class BlockGenerator(storageLevel: StorageLevel) extends Serializable with Logging { - case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) + case class Block(id: StreamBlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 7f3cd2f8eb1fd..9c69a2a4e21f5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -29,6 +29,7 @@ import org.apache.spark.streaming.Time */ case class BatchInfo( batchTime: Time, + receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]], submissionTime: Long, processingStartTime: Option[Long], processingEndTime: Option[Long] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 92d885c4bc5a5..e564eccba2df5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -201,7 +201,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { logInfo("Batches to reschedule (" + timesToReschedule.size + " batches): " + timesToReschedule.mkString(", ")) timesToReschedule.foreach(time => - jobScheduler.runJobs(time, graph.generateJobs(time)) + jobScheduler.submitJobSet(JobSet(time, graph.generateJobs(time))) ) // Restart the timer @@ -214,7 +214,12 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { case Success(jobs) => - jobScheduler.runJobs(time, jobs) + val receivedBlockInfo = graph.getNetworkInputStreams.map { stream => + val streamId = stream.id + val receivedBlockInfo = stream.getReceivedBlockInfo(time) + (streamId, receivedBlockInfo) + }.toMap + jobScheduler.submitJobSet(JobSet(time, jobs, receivedBlockInfo)) case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 04e0a6a283cfb..d9ada99b472ac 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -100,14 +100,13 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { logInfo("Stopped JobScheduler") } - def runJobs(time: Time, jobs: Seq[Job]) { - if (jobs.isEmpty) { - logInfo("No jobs added for time " + time) + def submitJobSet(jobSet: JobSet) { + if (jobSet.jobs.isEmpty) { + logInfo("No jobs added for time " + jobSet.time) } else { - val jobSet = new JobSet(time, jobs) - jobSets.put(time, jobSet) + jobSets.put(jobSet.time, jobSet) jobSet.jobs.foreach(job => jobExecutor.execute(new JobHandler(job))) - logInfo("Added jobs for time " + time) + logInfo("Added jobs for time " + jobSet.time) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala index fcf303aee6cd7..a69d74362173e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -24,7 +24,11 @@ import org.apache.spark.streaming.Time * belong to the same batch. */ private[streaming] -case class JobSet(time: Time, jobs: Seq[Job]) { +case class JobSet( + time: Time, + jobs: Seq[Job], + receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]] = Map.empty + ) { private val incompleteJobs = new HashSet[Job]() private val submissionTime = System.currentTimeMillis() // when this jobset was submitted @@ -60,6 +64,7 @@ case class JobSet(time: Time, jobs: Seq[Job]) { def toBatchInfo: BatchInfo = { new BatchInfo( time, + receivedBlockInfo, submissionTime, if (processingStartTime >= 0 ) Some(processingStartTime) else None, if (processingEndTime >= 0 ) Some(processingEndTime) else None diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index 067e804202236..a1e6f5176825a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -17,20 +17,42 @@ package org.apache.spark.streaming.scheduler -import scala.collection.mutable.{HashMap, Queue, SynchronizedMap} +import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} import akka.actor._ + import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.SparkContext._ -import org.apache.spark.storage.BlockId +import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.dstream.{NetworkReceiver, StopReceiver} import org.apache.spark.util.AkkaUtils +/** Information about receiver */ +case class ReceiverInfo(streamId: Int, typ: String, location: String) { + override def toString = s"$typ-$streamId" +} + +/** Information about blocks received by the network receiver */ +case class ReceivedBlockInfo( + streamId: Int, + blockId: StreamBlockId, + numRecords: Long, + metadata: Any + ) + +/** + * Messages used by the NetworkReceiver and the NetworkInputTracker to communicate + * with each other. + */ private[streaming] sealed trait NetworkInputTrackerMessage -private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) - extends NetworkInputTrackerMessage -private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) +private[streaming] case class RegisterReceiver( + streamId: Int, + typ: String, + host: String, + receiverActor: ActorRef + ) extends NetworkInputTrackerMessage +private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) extends NetworkInputTrackerMessage private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage @@ -47,9 +69,10 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) val receiverExecutor = new ReceiverExecutor() val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] - val receivedBlockIds = new HashMap[Int, Queue[BlockId]] with SynchronizedMap[Int, Queue[BlockId]] + val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] + with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] val timeout = AkkaUtils.askTimeout(ssc.conf) - + val listenerBus = ssc.scheduler.listenerBus // actor is created when generator starts. // This not being null means the tracker has been started and not stopped @@ -83,12 +106,32 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } } + /** Return all the blocks received from a receiver. */ + def getReceivedBlockInfo(streamId: Int): Array[ReceivedBlockInfo] = { + val receivedBlockInfo = getReceivedBlockInfoQueue(streamId).dequeueAll(x => true) + logInfo("Stream " + streamId + " received " + receivedBlockInfo.size + " blocks") + receivedBlockInfo.toArray + } + + private def getReceivedBlockInfoQueue(streamId: Int) = { + receivedBlockInfo.getOrElseUpdate(streamId, new SynchronizedQueue[ReceivedBlockInfo]) + } + /** Register a receiver */ - def registerReceiver(streamId: Int, receiverActor: ActorRef, sender: ActorRef) { + def registerReceiver( + streamId: Int, + typ: String, + host: String, + receiverActor: ActorRef, + sender: ActorRef + ) { if (!networkInputStreamMap.contains(streamId)) { throw new Exception("Register received for unexpected id " + streamId) } receiverInfo += ((streamId, receiverActor)) + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted( + ReceiverInfo(streamId, typ, host) + )) logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) } @@ -98,35 +141,26 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { logError("Deregistered receiver for network stream " + streamId + " with message:\n" + message) } - /** Get all the received blocks for the given stream. */ - def getBlocks(streamId: Int, time: Time): Array[BlockId] = { - val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]()) - val result = queue.dequeueAll(x => true).toArray - logInfo("Stream " + streamId + " received " + result.size + " blocks") - result - } - /** Add new blocks for the given stream */ - def addBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) = { - val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]) - queue ++= blockIds - networkInputStreamMap(streamId).addMetadata(metadata) - logDebug("Stream " + streamId + " received new blocks: " + blockIds.mkString("[", ", ", "]")) + def addBlocks(receivedBlockInfo: ReceivedBlockInfo) { + getReceivedBlockInfoQueue(receivedBlockInfo.streamId) += receivedBlockInfo + logDebug("Stream " + receivedBlockInfo.streamId + " received new blocks: " + + receivedBlockInfo.blockId) } /** Check if any blocks are left to be processed */ def hasMoreReceivedBlockIds: Boolean = { - !receivedBlockIds.forall(_._2.isEmpty) + !receivedBlockInfo.values.forall(_.isEmpty) } /** Actor to receive messages from the receivers. */ private class NetworkInputTrackerActor extends Actor { def receive = { - case RegisterReceiver(streamId, receiverActor) => - registerReceiver(streamId, receiverActor, sender) + case RegisterReceiver(streamId, typ, host, receiverActor) => + registerReceiver(streamId, typ, host, receiverActor, sender) sender ! true - case AddBlocks(streamId, blockIds, metadata) => - addBlocks(streamId, blockIds, metadata) + case AddBlock(receivedBlockInfo) => + addBlocks(receivedBlockInfo) case DeregisterReceiver(streamId, message) => deregisterReceiver(streamId, message) sender ! true diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 461ea3506477f..5db40ebbeb1de 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -23,8 +23,11 @@ import org.apache.spark.util.Distribution /** Base trait for events related to StreamingListener */ sealed trait StreamingListenerEvent +case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent +case class StreamingListenerReceiverStarted(receiverInfo: ReceiverInfo) + extends StreamingListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object StreamingListenerShutdown extends StreamingListenerEvent @@ -34,14 +37,17 @@ private[scheduler] case object StreamingListenerShutdown extends StreamingListen * computation. */ trait StreamingListener { - /** - * Called when processing of a batch has completed - */ + + /** Called when a receiver has been started */ + def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { } + + /** Called when a batch of jobs has been submitted for processing. */ + def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { } + + /** Called when processing of a batch of jobs has completed. */ def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } - /** - * Called when processing of a batch has started - */ + /** Called when processing of a batch of jobs has started. */ def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala index 18811fc2b01d8..ea03dfc7bfeea 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala @@ -38,6 +38,10 @@ private[spark] class StreamingListenerBus() extends Logging { while (true) { val event = eventQueue.take event match { + case receiverStarted: StreamingListenerReceiverStarted => + listeners.foreach(_.onReceiverStarted(receiverStarted)) + case batchSubmitted: StreamingListenerBatchSubmitted => + listeners.foreach(_.onBatchSubmitted(batchSubmitted)) case batchStarted: StreamingListenerBatchStarted => listeners.foreach(_.onBatchStarted(batchStarted)) case batchCompleted: StreamingListenerBatchCompleted => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala new file mode 100644 index 0000000000000..8b025b09ed34d --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.ui + +import org.apache.spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.scheduler._ +import scala.collection.mutable.{Queue, HashMap} +import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted +import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted +import org.apache.spark.streaming.scheduler.BatchInfo +import org.apache.spark.streaming.scheduler.ReceiverInfo +import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted +import org.apache.spark.util.Distribution + + +private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends StreamingListener { + + private val waitingBatchInfos = new HashMap[Time, BatchInfo] + private val runningBatchInfos = new HashMap[Time, BatchInfo] + private val completedaBatchInfos = new Queue[BatchInfo] + private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) + private var totalCompletedBatches = 0L + private val receiverInfos = new HashMap[Int, ReceiverInfo] + + val batchDuration = ssc.graph.batchDuration.milliseconds + + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) = { + synchronized { + receiverInfos.put(receiverStarted.receiverInfo.streamId, receiverStarted.receiverInfo) + } + } + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) = synchronized { + runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) = synchronized { + runningBatchInfos(batchStarted.batchInfo.batchTime) = batchStarted.batchInfo + waitingBatchInfos.remove(batchStarted.batchInfo.batchTime) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) = synchronized { + waitingBatchInfos.remove(batchCompleted.batchInfo.batchTime) + runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) + completedaBatchInfos.enqueue(batchCompleted.batchInfo) + if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() + totalCompletedBatches += 1L + } + + def numNetworkReceivers = synchronized { + ssc.graph.getNetworkInputStreams().size + } + + def numTotalCompletedBatches: Long = synchronized { + totalCompletedBatches + } + + def numUnprocessedBatches: Long = synchronized { + waitingBatchInfos.size + runningBatchInfos.size + } + + def waitingBatches: Seq[BatchInfo] = synchronized { + waitingBatchInfos.values.toSeq + } + + def runningBatches: Seq[BatchInfo] = synchronized { + runningBatchInfos.values.toSeq + } + + def retainedCompletedBatches: Seq[BatchInfo] = synchronized { + completedaBatchInfos.toSeq + } + + def processingDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.processingDelay) + } + + def schedulingDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.schedulingDelay) + } + + def totalDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.totalDelay) + } + + def receivedRecordsDistributions: Map[Int, Option[Distribution]] = synchronized { + val latestBatchInfos = retainedBatches.reverse.take(batchInfoLimit) + val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) + (0 until numNetworkReceivers).map { receiverId => + val blockInfoOfParticularReceiver = latestBlockInfos.map { batchInfo => + batchInfo.get(receiverId).getOrElse(Array.empty) + } + val recordsOfParticularReceiver = blockInfoOfParticularReceiver.map { blockInfo => + // calculate records per second for each batch + blockInfo.map(_.numRecords).sum.toDouble * 1000 / batchDuration + } + val distributionOption = Distribution(recordsOfParticularReceiver) + (receiverId, distributionOption) + }.toMap + } + + def lastReceivedBatchRecords: Map[Int, Long] = { + val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo) + lastReceivedBlockInfoOption.map { lastReceivedBlockInfo => + (0 until numNetworkReceivers).map { receiverId => + (receiverId, lastReceivedBlockInfo(receiverId).map(_.numRecords).sum) + }.toMap + }.getOrElse { + (0 until numNetworkReceivers).map(receiverId => (receiverId, 0L)).toMap + } + } + + def receiverInfo(receiverId: Int): Option[ReceiverInfo] = { + receiverInfos.get(receiverId) + } + + def lastCompletedBatch: Option[BatchInfo] = { + completedaBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption + } + + def lastReceivedBatch: Option[BatchInfo] = { + retainedBatches.lastOption + } + + private def retainedBatches: Seq[BatchInfo] = synchronized { + (waitingBatchInfos.values.toSeq ++ + runningBatchInfos.values.toSeq ++ completedaBatchInfos).sortBy(_.batchTime)(Time.ordering) + } + + private def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { + Distribution(completedaBatchInfos.flatMap(getMetric(_)).map(_.toDouble)) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala new file mode 100644 index 0000000000000..6607437db560a --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.ui + +import java.util.Calendar +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.Logging +import org.apache.spark.ui._ +import org.apache.spark.ui.UIUtils._ +import org.apache.spark.util.Distribution + +/** Page for Spark Web UI that shows statistics of a streaming job */ +private[ui] class StreamingPage(parent: StreamingTab) + extends WebUIPage("") with Logging { + + private val listener = parent.listener + private val startTime = Calendar.getInstance().getTime() + private val emptyCell = "-" + + /** Render the page */ + def render(request: HttpServletRequest): Seq[Node] = { + val content = + generateBasicStats() ++

      ++ +

      Statistics over last {listener.retainedCompletedBatches.size} processed batches

      ++ + generateNetworkStatsTable() ++ + generateBatchStatsTable() + UIUtils.headerSparkPage( + content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000)) + } + + /** Generate basic stats of the streaming program */ + private def generateBasicStats(): Seq[Node] = { + val timeSinceStart = System.currentTimeMillis() - startTime.getTime +
        +
      • + Started at: {startTime.toString} +
      • +
      • + Time since start: {formatDurationVerbose(timeSinceStart)} +
      • +
      • + Network receivers: {listener.numNetworkReceivers} +
      • +
      • + Batch interval: {formatDurationVerbose(listener.batchDuration)} +
      • +
      • + Processed batches: {listener.numTotalCompletedBatches} +
      • +
      • + Waiting batches: {listener.numUnprocessedBatches} +
      • +
      + } + + /** Generate stats of data received over the network the streaming program */ + private def generateNetworkStatsTable(): Seq[Node] = { + val receivedRecordDistributions = listener.receivedRecordsDistributions + val lastBatchReceivedRecord = listener.lastReceivedBatchRecords + val table = if (receivedRecordDistributions.size > 0) { + val headerRow = Seq( + "Receiver", + "Location", + "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", + "Minimum rate\n[records/sec]", + "25th percentile rate\n[records/sec]", + "Median rate\n[records/sec]", + "75th percentile rate\n[records/sec]", + "Maximum rate\n[records/sec]" + ) + val dataRows = (0 until listener.numNetworkReceivers).map { receiverId => + val receiverInfo = listener.receiverInfo(receiverId) + val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") + val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) + val receiverLastBatchRecords = formatDurationVerbose(lastBatchReceivedRecord(receiverId)) + val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => + d.getQuantiles().map(r => formatDurationVerbose(r.toLong)) + }.getOrElse { + Seq(emptyCell, emptyCell, emptyCell, emptyCell, emptyCell) + } + Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats + } + Some(listingTable(headerRow, dataRows)) + } else { + None + } + + val content = +
      Network Input Statistics
      ++ +
      {table.getOrElse("No network receivers")}
      + + content + } + + /** Generate stats of batch jobs of the streaming program */ + private def generateBatchStatsTable(): Seq[Node] = { + val numBatches = listener.retainedCompletedBatches.size + val lastCompletedBatch = listener.lastCompletedBatch + val table = if (numBatches > 0) { + val processingDelayQuantilesRow = { + Seq( + "Processing Time", + formatDurationOption(lastCompletedBatch.flatMap(_.processingDelay)) + ) ++ getQuantiles(listener.processingDelayDistribution) + } + val schedulingDelayQuantilesRow = { + Seq( + "Scheduling Delay", + formatDurationOption(lastCompletedBatch.flatMap(_.schedulingDelay)) + ) ++ getQuantiles(listener.schedulingDelayDistribution) + } + val totalDelayQuantilesRow = { + Seq( + "Total Delay", + formatDurationOption(lastCompletedBatch.flatMap(_.totalDelay)) + ) ++ getQuantiles(listener.totalDelayDistribution) + } + val headerRow = Seq("Metric", "Last batch", "Minimum", "25th percentile", + "Median", "75th percentile", "Maximum") + val dataRows: Seq[Seq[String]] = Seq( + processingDelayQuantilesRow, + schedulingDelayQuantilesRow, + totalDelayQuantilesRow + ) + Some(listingTable(headerRow, dataRows)) + } else { + None + } + + val content = +
      Batch Processing Statistics
      ++ +
      +
        + {table.getOrElse("No statistics have been generated yet.")} +
      +
      + + content + } + + + /** + * Returns a human-readable string representing a duration such as "5 second 35 ms" + */ + private def formatDurationOption(msOption: Option[Long]): String = { + msOption.map(formatDurationVerbose).getOrElse(emptyCell) + } + + /** Get quantiles for any time distribution */ + private def getQuantiles(timeDistributionOption: Option[Distribution]) = { + timeDistributionOption.get.getQuantiles().map { ms => formatDurationVerbose(ms.toLong) } + } + + /** Generate HTML table from string data */ + private def listingTable(headers: Seq[String], data: Seq[Seq[String]]) = { + def generateDataRow(data: Seq[String]): Seq[Node] = { + {data.map(d => {d})} + } + UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) + } +} + diff --git a/core/src/test/scala/org/apache/spark/SparkUISuite.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala similarity index 58% rename from core/src/test/scala/org/apache/spark/SparkUISuite.scala rename to streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index d0d119c15081d..51448d15c6516 100644 --- a/core/src/test/scala/org/apache/spark/SparkUISuite.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -15,21 +15,22 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.streaming.ui -import java.net.URI +import org.apache.spark.Logging +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.ui.WebUITab -import org.scalatest.FunSuite +/** Spark Web UI tab that shows statistics of a streaming job */ +private[spark] class StreamingTab(ssc: StreamingContext) + extends WebUITab(ssc.sc.ui, "streaming") with Logging { -class SparkUISuite extends FunSuite with SharedSparkContext { + val parent = ssc.sc.ui + val appName = parent.appName + val basePath = parent.basePath + val listener = new StreamingJobProgressListener(ssc) - test("verify appUIAddress contains the scheme") { - val uiAddress = sc.ui.appUIAddress - assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) - } - - test("verify appUIAddress contains the port") { - val splitUIAddress = sc.ui.appUIAddress.split(':') - assert(splitUIAddress(2).toInt == sc.ui.boundPort) - } + ssc.addStreamingListener(listener) + attachPage(new StreamingPage(this)) + parent.attachTab(this) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 389b23d4d5e4b..952511d411a8e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -239,11 +239,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { /** This is a server to test the network input stream */ -class TestServer() extends Logging { +class TestServer(portToBind: Int = 0) extends Logging { val queue = new ArrayBlockingQueue[String](100) - val serverSocket = new ServerSocket(0) + val serverSocket = new ServerSocket(portToBind) val servingThread = new Thread() { override def run() { @@ -282,7 +282,7 @@ class TestServer() extends Logging { def start() { servingThread.start() } - def send(msg: String) { queue.add(msg) } + def send(msg: String) { queue.put(msg) } def stop() { servingThread.interrupt() } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 9cc27ef7f03b5..efd0d22ecb57a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -161,7 +161,6 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } } - test("stop only streaming context") { ssc = new StreamingContext(master, appName, batchDuration) sc = ssc.sparkContext diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala new file mode 100644 index 0000000000000..35538ec188f67 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import scala.io.Source + +import org.scalatest.FunSuite +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +class UISuite extends FunSuite { + + test("streaming tab in spark UI") { + val ssc = new StreamingContext("local", "test", Seconds(1)) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(ssc.sparkContext.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + // test if streaming tab exist + assert(html.toLowerCase.contains("streaming")) + // test if other Spark tabs still exist + assert(html.toLowerCase.contains("stages")) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL( + ssc.sparkContext.ui.appUIAddress.stripSuffix("/") + "/streaming").mkString + assert(html.toLowerCase.contains("batch")) + assert(html.toLowerCase.contains("network")) + } + } +} From c2d160fbee2ef90a7683d9771f2f632b68d74aef Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 12 Apr 2014 16:33:38 -0700 Subject: [PATCH 22/61] [Fix #204] Update out-dated comments This PR is self-explanatory. Author: Andrew Or Closes #381 from andrewor14/master and squashes the following commits: 3e8dde2 [Andrew Or] Fix comments for #204 --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +----- .../org/apache/spark/scheduler/ReplayListenerBus.scala | 4 ++-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a764c174d562c..5a36e6f5c19a9 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1191,11 +1191,7 @@ class SparkContext(config: SparkConf) extends Logging { listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) } - /** - * Post the application end event to all listeners immediately, rather than adding it - * to the event queue for it to be asynchronously processed eventually. Otherwise, a race - * condition exists in which the listeners may stop before this event has been propagated. - */ + /** Post the application end event */ private def postApplicationEnd() { listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index f868e772cf58a..f89724d4ea196 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -31,8 +31,8 @@ import org.apache.spark.util.JsonProtocol /** * A SparkListenerBus that replays logged events from persisted storage. * - * This class expects files to be appropriately prefixed as specified in EventLoggingListener. - * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. + * This assumes the given paths are valid log files, where each line can be deserialized into + * exactly one SparkListenerEvent. */ private[spark] class ReplayListenerBus( logPaths: Seq[Path], From ca11919e6e97a62eb3e3ce882ffa29eae36f50f7 Mon Sep 17 00:00:00 2001 From: Bharath Bhushan Date: Sat, 12 Apr 2014 20:52:29 -0700 Subject: [PATCH 23/61] [SPARK-1403] Move the class loader creation back to where it was in 0.9.0 [SPARK-1403] I investigated why spark 0.9.0 loads fine on mesos while spark 1.0.0 fails. What I found was that in SparkEnv.scala, while creating the SparkEnv object, the current thread's classloader is null. But in 0.9.0, at the same place, it is set to org.apache.spark.repl.ExecutorClassLoader . I saw that https://github.com/apache/spark/commit/7edbea41b43e0dc11a2de156be220db8b7952d01 moved it to it current place. I moved it back and saw that 1.0.0 started working fine on mesos. I just created a minimal patch that allows me to run spark on mesos correctly. It seems like SecurityManager's creation needs to be taken into account for a correct fix. Also moving the creation of the serializer out of SparkEnv might be a part of the right solution. PTAL. Author: Bharath Bhushan Closes #322 from manku-timma/spark-1403 and squashes the following commits: 606c2b9 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 ec8f870 [Bharath Bhushan] revert the logger change for java 6 compatibility as PR 334 is doing it 728beca [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 044027d [Bharath Bhushan] fix compile error 6f260a4 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 b3a053f [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 04b9662 [Bharath Bhushan] add missing line 4803c19 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 f3c9a14 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 42d3d6a [Bharath Bhushan] used code fragment from @ueshin to fix the problem in a better way 89109d7 [Bharath Bhushan] move the class loader creation back to where it was in 0.9.0 --- .../spark/executor/MesosExecutorBackend.scala | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 6fc702fdb1512..df36a06485c77 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -50,13 +50,21 @@ private[spark] class MesosExecutorBackend executorInfo: ExecutorInfo, frameworkInfo: FrameworkInfo, slaveInfo: SlaveInfo) { - logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) - this.driver = driver - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) - executor = new Executor( - executorInfo.getExecutorId.getValue, - slaveInfo.getHostname, - properties) + val cl = Thread.currentThread.getContextClassLoader + try { + // Work around for SPARK-1480 + Thread.currentThread.setContextClassLoader(getClass.getClassLoader) + logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) + this.driver = driver + val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) + executor = new Executor( + executorInfo.getExecutorId.getValue, + slaveInfo.getHostname, + properties) + } finally { + // Work around for SPARK-1480 + Thread.currentThread.setContextClassLoader(cl) + } } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { From 4bc07eebbf5e2ea0c0b6f1642049515025d88d07 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 13 Apr 2014 08:58:37 -0700 Subject: [PATCH 24/61] SPARK-1480: Clean up use of classloaders The Spark codebase is a bit fast-and-loose when accessing classloaders and this has caused a few bugs to surface in master. This patch defines some utility methods for accessing classloaders. This makes the intention when accessing a classloader much more explicit in the code and fixes a few cases where the wrong one was chosen. case (a) -> We want the classloader that loaded Spark case (b) -> We want the context class loader, or if not present, we want (a) This patch provides a better fix for SPARK-1403 (https://issues.apache.org/jira/browse/SPARK-1403) than the current work around, which it reverts. It also fixes a previously unreported bug that the `./spark-submit` script did not work for running with `local` master. It didn't work because the executor classloader did not properly delegate to the context class loader (if it is defined) and in local mode the context class loader is set by the `./spark-submit` script. A unit test is added for that case. Author: Patrick Wendell Closes #398 from pwendell/class-loaders and squashes the following commits: b4a1a58 [Patrick Wendell] Minor clean up 14f1272 [Patrick Wendell] SPARK-1480: Clean up use of classloaders --- .../main/scala/org/apache/spark/Logging.scala | 4 +-- .../org/apache/spark/executor/Executor.scala | 6 ++-- .../spark/executor/MesosExecutorBackend.scala | 22 ++++--------- .../apache/spark/metrics/MetricsConfig.scala | 3 +- .../apache/spark/scheduler/ResultTask.scala | 1 - .../spark/scheduler/SchedulableBuilder.scala | 3 +- .../spark/scheduler/TaskResultGetter.scala | 4 +-- .../spark/serializer/JavaSerializer.scala | 3 +- .../org/apache/spark/ui/JettyUtils.scala | 3 +- .../scala/org/apache/spark/util/Utils.scala | 15 +++++++++ .../ExecutorURLClassLoaderSuite.scala | 32 +++++++++++++++++-- .../org/apache/spark/repl/SparkILoop.scala | 7 ++-- .../spark/sql/catalyst/util/package.scala | 4 ++- .../compression/compressionSchemes.scala | 3 +- .../sql/execution/SparkSqlSerializer.scala | 3 +- 15 files changed, 78 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 9d429dceeb858..50d8e93e1f0d7 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -22,6 +22,7 @@ import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -115,8 +116,7 @@ trait Logging { val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized && usingLog4j) { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" - val classLoader = this.getClass.getClassLoader - Option(classLoader.getResource(defaultLogProps)) match { + Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match { case Some(url) => PropertyConfigurator.configure(url) log.info(s"Using Spark's default log4j profile: $defaultLogProps") diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c12bd922d40e4..f89b2bffd1676 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -292,7 +292,7 @@ private[spark] class Executor( * created by the interpreter to the search path */ private def createClassLoader(): MutableURLClassLoader = { - val loader = this.getClass.getClassLoader + val currentLoader = Utils.getContextOrSparkClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. @@ -301,8 +301,8 @@ private[spark] class Executor( }.toArray val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false) userClassPathFirst match { - case true => new ChildExecutorURLClassLoader(urls, loader) - case false => new ExecutorURLClassLoader(urls, loader) + case true => new ChildExecutorURLClassLoader(urls, currentLoader) + case false => new ExecutorURLClassLoader(urls, currentLoader) } } diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index df36a06485c77..6fc702fdb1512 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -50,21 +50,13 @@ private[spark] class MesosExecutorBackend executorInfo: ExecutorInfo, frameworkInfo: FrameworkInfo, slaveInfo: SlaveInfo) { - val cl = Thread.currentThread.getContextClassLoader - try { - // Work around for SPARK-1480 - Thread.currentThread.setContextClassLoader(getClass.getClassLoader) - logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) - this.driver = driver - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) - executor = new Executor( - executorInfo.getExecutorId.getValue, - slaveInfo.getHostname, - properties) - } finally { - // Work around for SPARK-1480 - Thread.currentThread.setContextClassLoader(cl) - } + logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) + this.driver = driver + val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) + executor = new Executor( + executorInfo.getExecutorId.getValue, + slaveInfo.getHostname, + properties) } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index 3e3e18c3537d0..1b7a5d1f1980a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -24,6 +24,7 @@ import scala.collection.mutable import scala.util.matching.Regex import org.apache.spark.Logging +import org.apache.spark.util.Utils private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { @@ -50,7 +51,7 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi try { is = configFile match { case Some(f) => new FileInputStream(f) - case None => getClass.getClassLoader.getResourceAsStream(METRICS_CONF) + case None => Utils.getSparkClassLoader.getResourceAsStream(METRICS_CONF) } if (is != null) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 083fb895d8696..0b381308b61ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -54,7 +54,6 @@ private[spark] object ResultTask { def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = { - val loader = Thread.currentThread.getContextClassLoader val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) val ser = SparkEnv.get.closureSerializer.newInstance() val objIn = ser.deserializeStream(in) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index e4eced383c3a5..6c5827f75e636 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -23,6 +23,7 @@ import java.util.{NoSuchElementException, Properties} import scala.xml.XML import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.Utils /** * An interface to build Schedulable tree @@ -72,7 +73,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) schedulerAllocFile.map { f => new FileInputStream(f) }.getOrElse { - getClass.getClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) + Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index cb4ad4ae9350c..c9ad2b151daf0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -85,13 +85,13 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul try { if (serializedData != null && serializedData.limit() > 0) { reason = serializer.get().deserialize[TaskEndReason]( - serializedData, getClass.getClassLoader) + serializedData, Utils.getSparkClassLoader) } } catch { case cnd: ClassNotFoundException => // Log an error but keep going here -- the task failed, so not catastropic if we can't // deserialize the reason. - val loader = Thread.currentThread.getContextClassLoader + val loader = Utils.getContextOrSparkClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) case ex: Throwable => {} diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 5e5883554fcc1..e9163deaf2036 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -23,6 +23,7 @@ import java.nio.ByteBuffer import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.Utils private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) extends SerializationStream { @@ -86,7 +87,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } def deserializeStream(s: InputStream): DeserializationStream = { - new JavaDeserializationStream(s, Thread.currentThread.getContextClassLoader) + new JavaDeserializationStream(s, Utils.getContextOrSparkClassLoader) } def deserializeStream(s: InputStream, loader: ClassLoader): DeserializationStream = { diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 62a4e3d0f6a42..3ae147a36c8a4 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -33,6 +33,7 @@ import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.util.Utils /** * Utilities for launching a web server using Jetty's HTTP Server class @@ -124,7 +125,7 @@ private[spark] object JettyUtils extends Logging { contextHandler.setInitParameter("org.eclipse.jetty.servlet.Default.gzip", "false") val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) - Option(getClass.getClassLoader.getResource(resourceBase)) match { + Option(Utils.getSparkClassLoader.getResource(resourceBase)) match { case Some(res) => holder.setInitParameter("resourceBase", res.toString) case None => diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 166f48ce7342e..a3af4e7b91692 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -116,6 +116,21 @@ private[spark] object Utils extends Logging { } } + /** + * Get the ClassLoader which loaded Spark. + */ + def getSparkClassLoader = getClass.getClassLoader + + /** + * Get the Context ClassLoader on this thread or, if not present, the ClassLoader that + * loaded Spark. + * + * This should be used whenever passing a ClassLoader to Class.ForName or finding the currently + * active loader when setting up ClassLoader delegation chains. + */ + def getContextOrSparkClassLoader = + Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader) + /** * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. */ diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala index c40cfc0696fce..e2050e95a1b88 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala @@ -17,12 +17,12 @@ package org.apache.spark.executor -import java.io.File import java.net.URLClassLoader import org.scalatest.FunSuite -import org.apache.spark.TestUtils +import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, TestUtils} +import org.apache.spark.util.Utils class ExecutorURLClassLoaderSuite extends FunSuite { @@ -63,5 +63,33 @@ class ExecutorURLClassLoaderSuite extends FunSuite { } } + test("driver sets context class loader in local mode") { + // Test the case where the driver program sets a context classloader and then runs a job + // in local mode. This is what happens when ./spark-submit is called with "local" as the + // master. + val original = Thread.currentThread().getContextClassLoader + val className = "ClassForDriverTest" + val jar = TestUtils.createJarWithClasses(Seq(className)) + val contextLoader = new URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) + Thread.currentThread().setContextClassLoader(contextLoader) + + val sc = new SparkContext("local", "driverLoaderTest") + + try { + sc.makeRDD(1 to 5, 2).mapPartitions { x => + val loader = Thread.currentThread().getContextClassLoader + Class.forName(className, true, loader).newInstance() + Seq().iterator + }.count() + } + catch { + case e: SparkException if e.getMessage.contains("ClassNotFoundException") => + fail("Local executor could not find class", e) + case t: Throwable => fail("Unexpected exception ", t) + } + + sc.stop() + Thread.currentThread().setContextClassLoader(original) + } } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 5a367b6bb79de..beb40e87024bd 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -39,6 +39,7 @@ import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} import org.apache.spark.Logging import org.apache.spark.SparkConf import org.apache.spark.SparkContext +import org.apache.spark.util.Utils /** The Scala interactive shell. It provides a read-eval-print loop * around the Interpreter class. @@ -130,7 +131,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def history = in.history /** The context class loader at the time this object was created */ - protected val originalClassLoader = Thread.currentThread.getContextClassLoader + protected val originalClassLoader = Utils.getContextOrSparkClassLoader // classpath entries added via :cp var addedClasspath: String = "" @@ -177,7 +178,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, override lazy val formatting = new Formatting { def prompt = SparkILoop.this.prompt } - override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) + override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) } /** Create a new interpreter. */ @@ -871,7 +872,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - val m = u.runtimeMirror(getClass.getClassLoader) + val m = u.runtimeMirror(Utils.getSparkClassLoader) private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = u.TypeTag[T]( m, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index a001d953592db..49fc4f70fdfae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File} +import org.apache.spark.util.{Utils => SparkUtils} + package object util { /** * Returns a path to a temporary file that probably does not exist. @@ -54,7 +56,7 @@ package object util { def resourceToString( resource:String, encoding: String = "UTF-8", - classLoader: ClassLoader = this.getClass.getClassLoader) = { + classLoader: ClassLoader = SparkUtils.getSparkClassLoader) = { val inStream = classLoader.getResourceAsStream(resource) val outStream = new ByteArrayOutputStream try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index df8220b556edd..e92cf5ac4f9df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -26,6 +26,7 @@ import scala.reflect.runtime.universe.runtimeMirror import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar._ +import org.apache.spark.util.Utils private[sql] case object PassThrough extends CompressionScheme { override val typeId = 0 @@ -254,7 +255,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { private val dictionary = { // TODO Can we clean up this mess? Maybe move this to `DataType`? implicit val classTag = { - val mirror = runtimeMirror(getClass.getClassLoader) + val mirror = runtimeMirror(Utils.getSparkClassLoader) ClassTag[T#JvmType](mirror.runtimeClass(columnType.scalaTag.tpe)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index d8e1b970c1d88..c30ae5bcc02d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -25,6 +25,7 @@ import com.esotericsoftware.kryo.{Serializer, Kryo} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair +import org.apache.spark.util.Utils class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { @@ -44,7 +45,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]]) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) kryo.setReferences(false) - kryo.setClassLoader(this.getClass.getClassLoader) + kryo.setClassLoader(Utils.getSparkClassLoader) kryo } } From 037fe4d2ba01be5610baa3dd9c5c9d3a5e5e1064 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Sun, 13 Apr 2014 13:18:52 -0700 Subject: [PATCH 25/61] [SPARK-1415] Hadoop min split for wholeTextFiles() JIRA issue [here](https://issues.apache.org/jira/browse/SPARK-1415). New Hadoop API of `InputFormat` does not provide the `minSplits` parameter, which makes the API incompatible between `HadoopRDD` and `NewHadoopRDD`. The PR is for constructing compatible APIs. Though `minSplits` is deprecated by New Hadoop API, we think it is better to make APIs compatible here. **Note** that `minSplits` in `wholeTextFiles` could only be treated as a *suggestion*, the real number of splits may not be greater than `minSplits` due to `isSplitable()=false`. Author: Xusen Yin Closes #376 from yinxusen/hadoop-min-split and squashes the following commits: 76417f6 [Xusen Yin] refine comments c10af60 [Xusen Yin] refine comments and rewrite new class for wholeTextFile 766d05b [Xusen Yin] refine Java API and comments 4875755 [Xusen Yin] add minSplits for WholeTextFiles --- .../scala/org/apache/spark/SparkContext.scala | 17 ++++-- .../spark/api/java/JavaSparkContext.scala | 14 ++++- .../input/WholeTextFileInputFormat.scala | 14 +++++ .../org/apache/spark/rdd/NewHadoopRDD.scala | 60 +++++++++++++++---- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../WholeTextFileRecordReaderSuite.scala | 2 +- 6 files changed, 90 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5a36e6f5c19a9..456070fa7c5ef 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -454,14 +454,21 @@ class SparkContext(config: SparkConf) extends Logging { * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are preferred, as each file will be loaded fully in memory. + * @note Small files are preferred, large file is also allowable, but may cause bad performance. + * + * @param minSplits A suggestion value of the minimal splitting number for input data. */ - def wholeTextFiles(path: String): RDD[(String, String)] = { - newAPIHadoopFile( - path, + def wholeTextFiles(path: String, minSplits: Int = defaultMinSplits): RDD[(String, String)] = { + val job = new NewHadoopJob(hadoopConfiguration) + NewFileInputFormat.addInputPath(job, new Path(path)) + val updateConf = job.getConfiguration + new WholeTextFileRDD( + this, classOf[WholeTextFileInputFormat], classOf[String], - classOf[String]) + classOf[String], + updateConf, + minSplits) } /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 1e8242a2cbbce..7fbefe1cb0fb1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -177,7 +177,19 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are preferred, as each file will be loaded fully in memory. + * @note Small files are preferred, large file is also allowable, but may cause bad performance. + * + * @param minSplits A suggestion value of the minimal splitting number for input data. + */ + def wholeTextFiles(path: String, minSplits: Int): JavaPairRDD[String, String] = + new JavaPairRDD(sc.wholeTextFiles(path, minSplits)) + + /** + * Read a directory of text files from HDFS, a local file system (available on all nodes), or any + * Hadoop-supported file system URI. Each file is read as a single record and returned in a + * key-value pair, where the key is the path of each file, the value is the content of each file. + * + * @see `wholeTextFiles(path: String, minSplits: Int)`. */ def wholeTextFiles(path: String): JavaPairRDD[String, String] = new JavaPairRDD(sc.wholeTextFiles(path)) diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index 4887fb6b84eb2..80d055a89573b 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -17,6 +17,8 @@ package org.apache.spark.input +import scala.collection.JavaConversions._ + import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.mapreduce.JobContext @@ -44,4 +46,16 @@ private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[Str context, classOf[WholeTextFileRecordReader]) } + + /** + * Allow minSplits set by end-user in order to keep compatibility with old Hadoop API. + */ + def setMaxSplitSize(context: JobContext, minSplits: Int) { + val files = listStatus(context) + val totalLen = files.map { file => + if (file.isDir) 0L else file.getLen + }.sum + val maxSplitSize = Math.ceil(totalLen * 1.0 / (if (minSplits == 0) 1 else minSplits)).toLong + super.setMaxSplitSize(maxSplitSize) + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 2d8dfa5a1645a..8684b645bc361 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -24,11 +24,18 @@ import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} import org.apache.spark.annotation.DeveloperApi - -private[spark] -class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) +import org.apache.spark.input.WholeTextFileInputFormat +import org.apache.spark.InterruptibleIterator +import org.apache.spark.Logging +import org.apache.spark.Partition +import org.apache.spark.SerializableWritable +import org.apache.spark.{SparkContext, TaskContext} + +private[spark] class NewHadoopPartition( + rddId: Int, + val index: Int, + @transient rawSplit: InputSplit with Writable) extends Partition { val serializableHadoopSplit = new SerializableWritable(rawSplit) @@ -65,17 +72,19 @@ class NewHadoopRDD[K, V]( private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) // private val serializableConf = new SerializableWritable(conf) - private val jobtrackerId: String = { + private val jobTrackerId: String = { val formatter = new SimpleDateFormat("yyyyMMddHHmm") formatter.format(new Date()) } - @transient private val jobId = new JobID(jobtrackerId, id) + @transient protected val jobId = new JobID(jobTrackerId, id) override def getPartitions: Array[Partition] = { val inputFormat = inputFormatClass.newInstance - if (inputFormat.isInstanceOf[Configurable]) { - inputFormat.asInstanceOf[Configurable].setConf(conf) + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => } val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray @@ -91,11 +100,13 @@ class NewHadoopRDD[K, V]( val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value - val attemptId = newTaskAttemptID(jobtrackerId, id, isMap = true, split.index, 0) + val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance - if (format.isInstanceOf[Configurable]) { - format.asInstanceOf[Configurable].setConf(conf) + format match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => } val reader = format.createRecordReader( split.serializableHadoopSplit.value, hadoopAttemptContext) @@ -141,3 +152,30 @@ class NewHadoopRDD[K, V]( def getConf: Configuration = confBroadcast.value.value } +private[spark] class WholeTextFileRDD( + sc : SparkContext, + inputFormatClass: Class[_ <: WholeTextFileInputFormat], + keyClass: Class[String], + valueClass: Class[String], + @transient conf: Configuration, + minSplits: Int) + extends NewHadoopRDD[String, String](sc, inputFormatClass, keyClass, valueClass, conf) { + + override def getPartitions: Array[Partition] = { + val inputFormat = inputFormatClass.newInstance + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val jobContext = newJobContext(conf, jobId) + inputFormat.setMaxSplitSize(jobContext, minSplits) + val rawSplits = inputFormat.getSplits(jobContext).toArray + val result = new Array[Partition](rawSplits.size) + for (i <- 0 until rawSplits.size) { + result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + } + result + } +} + diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index ab2fdac553349..8d2e9f1846343 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -626,7 +626,7 @@ public void wholeTextFiles() throws IOException { container.put(tempDirName+"/part-00000", new Text(content1).toString()); container.put(tempDirName+"/part-00001", new Text(content2).toString()); - JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName); + JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName, 3); List> result = readRDD.collect(); for (Tuple2 res : result) { diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index e89b296d41026..33d6de9a76405 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -73,7 +73,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { createNativeFile(dir, filename, contents) } - val res = sc.wholeTextFiles(dir.toString).collect() + val res = sc.wholeTextFiles(dir.toString, 3).collect() assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, "Number of files read out does not fit with the actual value.") From 7dbca68e92416ec5f023c8807bb06470c01a6d3a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 14 Apr 2014 15:22:43 -0700 Subject: [PATCH 26/61] [BUGFIX] In-memory columnar storage bug fixes Fixed several bugs of in-memory columnar storage to make `HiveInMemoryCompatibilitySuite` pass. @rxin @marmbrus It is reasonable to include `HiveInMemoryCompatibilitySuite` in this PR, but I didn't, since it significantly increases test execution time. What do you think? **UPDATE** `HiveCompatibilitySuite` has been made to cache tables in memory. `HiveInMemoryCompatibilitySuite` was removed. Author: Cheng Lian Author: Michael Armbrust Closes #374 from liancheng/inMemBugFix and squashes the following commits: 6ad6d9b [Cheng Lian] Merged HiveCompatibilitySuite and HiveInMemoryCompatibilitySuite 5bdbfe7 [Cheng Lian] Revert 882c538 & 8426ddc, which introduced regression 882c538 [Cheng Lian] Remove attributes field from InMemoryColumnarTableScan 32cc9ce [Cheng Lian] Code style cleanup 99382bf [Cheng Lian] Enable compression by default 4390bcc [Cheng Lian] Report error for any Throwable in HiveComparisonTest d1df4fd [Michael Armbrust] Remove test tables that might always get created anyway? ab9e807 [Michael Armbrust] Fix the logged console version of failed test cases to use the new syntax. 1965123 [Michael Armbrust] Don't use coalesce for gathering all data to a single partition, as it does not work correctly with mutable rows. e36cdd0 [Michael Armbrust] Spelling. 2d0e168 [Michael Armbrust] Run Hive tests in-memory too. 6360723 [Cheng Lian] Made PreInsertionCasts support SparkLogicalPlan and InMemoryColumnarTableScan c9b0f6f [Cheng Lian] Let InsertIntoTable support InMemoryColumnarTableScan 9c8fc40 [Cheng Lian] Disable compression by default e619995 [Cheng Lian] Bug fix: incorrect byte order in CompressionScheme.columnHeaderSize 8426ddc [Cheng Lian] Bug fix: InMemoryColumnarTableScan should cache columns specified by the attributes argument 036cd09 [Cheng Lian] Clean up unused imports 44591a5 [Cheng Lian] Bug fix: NullableColumnAccessor.hasNext must take nulls into account 052bf41 [Cheng Lian] Bug fix: should only gather compressibility info for non-null values 95b3301 [Cheng Lian] Fixed bugs in IntegralDelta --- .../org/apache/spark/sql/SchemaRDD.scala | 2 +- .../sql/columnar/NullableColumnAccessor.scala | 2 + .../CompressibleColumnBuilder.scala | 4 +- .../compression/CompressionScheme.scala | 4 +- .../compression/compressionSchemes.scala | 20 ++++----- .../apache/spark/sql/execution/Exchange.scala | 9 +++- .../spark/sql/execution/SparkPlan.scala | 4 +- .../apache/spark/sql/CachedTableSuite.scala | 5 +-- ...scala => InMemoryColumnarQuerySuite.scala} | 12 +++++- .../NullableColumnAccessorSuite.scala | 4 ++ .../compression/IntegralDeltaSuite.scala | 15 +++++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 42 ++++++++++++------- .../spark/sql/hive/HiveStrategies.scala | 4 ++ .../org/apache/spark/sql/hive/TestHive.scala | 10 ++--- .../org/apache/spark/sql/hive/hiveUdfs.scala | 16 +++---- .../hive/execution/HiveComparisonTest.scala | 10 +---- .../execution/HiveCompatibilitySuite.scala | 12 +++++- 17 files changed, 109 insertions(+), 66 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/columnar/{ColumnarQuerySuite.scala => InMemoryColumnarQuerySuite.scala} (79%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 16da7fd92bffe..91500416eefaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -99,7 +99,7 @@ class SchemaRDD( def baseSchemaRDD = this // ========================================================================================= - // RDD functions: Copy the interal row representation so we present immutable data to users. + // RDD functions: Copy the internal row representation so we present immutable data to users. // ========================================================================================= override def compute(split: Partition, context: TaskContext): Iterator[Row] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala index 7d49ab07f7a53..b7f8826861a2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala @@ -54,4 +54,6 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { pos += 1 } + + abstract override def hasNext = seenNulls < nullCount || super.hasNext } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index fd3b1adf9687a..0f808f68f2eec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -65,7 +65,9 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] abstract override def appendFrom(row: Row, ordinal: Int) { super.appendFrom(row, ordinal) - gatherCompressibilityStats(row, ordinal) + if (!row.isNullAt(ordinal)) { + gatherCompressibilityStats(row, ordinal) + } } abstract override def build() = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index c605a8e4434e3..ba1810dd2ae66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.columnar.compression -import java.nio.ByteBuffer +import java.nio.{ByteOrder, ByteBuffer} import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} @@ -84,7 +84,7 @@ private[sql] object CompressionScheme { } def columnHeaderSize(columnBuffer: ByteBuffer): Int = { - val header = columnBuffer.duplicate() + val header = columnBuffer.duplicate().order(ByteOrder.nativeOrder) val nullCount = header.getInt(4) // Column type ID + null count + null positions 4 + 4 + 4 * nullCount diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index e92cf5ac4f9df..800009d3195e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -397,26 +397,27 @@ private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends Comp if (initial) { initial = false - prev = value _compressedSize += 1 + columnType.defaultSize } else { val (smallEnough, _) = byteSizedDelta(value, prev) _compressedSize += (if (smallEnough) 1 else 1 + columnType.defaultSize) } + + prev = value } override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[I]) = { to.putInt(typeId) if (from.hasRemaining) { - val prev = columnType.extract(from) - + var prev = columnType.extract(from) to.put(Byte.MinValue) columnType.append(prev, to) while (from.hasRemaining) { val current = columnType.extract(from) val (smallEnough, delta) = byteSizedDelta(current, prev) + prev = current if (smallEnough) { to.put(delta) @@ -443,13 +444,8 @@ private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends Comp override def next() = { val delta = buffer.get() - - if (delta > Byte.MinValue) { - addDelta(prev, delta) - } else { - prev = columnType.extract(buffer) - prev - } + prev = if (delta > Byte.MinValue) addDelta(prev, delta) else columnType.extract(buffer) + prev } override def hasNext = buffer.hasRemaining @@ -465,7 +461,7 @@ private[sql] case object IntDelta extends IntegralDelta[IntegerType.type] { override protected def byteSizedDelta(x: Int, y: Int): (Boolean, Byte) = { val delta = x - y - if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) } } @@ -478,6 +474,6 @@ private[sql] case object LongDelta extends IntegralDelta[LongType.type] { override protected def byteSizedDelta(x: Long, y: Long): (Boolean, Byte) = { val delta = x - y - if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 450c142c0baa4..070557e47c4c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -61,7 +61,14 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.map(_._1) case SinglePartition => - child.execute().coalesce(1, shuffle = true) + val rdd = child.execute().mapPartitions { iter => + val mutablePair = new MutablePair[Null, Row]() + iter.map(r => mutablePair.update(null, r)) + } + val partitioner = new HashPartitioner(1) + val shuffled = new ShuffledRDD[Null, Row, MutablePair[Null, Row]](rdd, partitioner) + shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") // TODO: Handle BroadcastPartitioning. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index daa423cb8ea1a..5d89697db5f99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -70,8 +70,8 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) - case InMemoryColumnarTableScan(output, child) => - InMemoryColumnarTableScan(output.map(_.newInstance), child) + case scan @ InMemoryColumnarTableScan(output, child) => + scan.copy(attributes = output.map(_.newInstance)) case _ => sys.error("Multiple instance of the same relation detected.") }).asInstanceOf[this.type] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 7c6a642278226..0331f90272a99 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -17,11 +17,10 @@ package org.apache.spark.sql -import org.scalatest.FunSuite import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.columnar.InMemoryColumnarTableScan +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.test.TestSQLContext class CachedTableSuite extends QueryTest { TestData // Load test tables. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala similarity index 79% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 2ed4cf2170f9d..16a13b8a74960 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.{QueryTest, TestData} +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.test.TestSQLContext -class ColumnarQuerySuite extends QueryTest { +class InMemoryColumnarQuerySuite extends QueryTest { import TestData._ import TestSQLContext._ @@ -32,6 +33,15 @@ class ColumnarQuerySuite extends QueryTest { checkAnswer(scan, testData.collect().toSeq) } + test("projection") { + val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + + checkAnswer(scan, testData.collect().map { + case Row(key: Int, value: String) => value -> key + }.toSeq) + } + test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 4a21eb6201a69..35ab14cbc353d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -68,12 +68,16 @@ class NullableColumnAccessorSuite extends FunSuite { val row = new GenericMutableRow(1) (0 until 4).foreach { _ => + assert(accessor.hasNext) accessor.extractTo(row, 0) assert(row(0) === randomRow(0)) + assert(accessor.hasNext) accessor.extractTo(row, 0) assert(row.isNullAt(0)) } + + assert(!accessor.hasNext) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 1390e5eef6106..ce419ca7269ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.IntegralType import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.columnar.ColumnarTestUtils._ class IntegralDeltaSuite extends FunSuite { testIntegralDelta(new IntColumnStats, INT, IntDelta) @@ -63,7 +64,7 @@ class IntegralDeltaSuite extends FunSuite { } else { val oneBoolean = columnType.defaultSize 1 + oneBoolean + deltas.map { - d => if (math.abs(d) < Byte.MaxValue) 1 else 1 + oneBoolean + d => if (math.abs(d) <= Byte.MaxValue) 1 else 1 + oneBoolean }.sum }) @@ -78,7 +79,7 @@ class IntegralDeltaSuite extends FunSuite { expectResult(input.head, "The first value is wrong")(columnType.extract(buffer)) (input.tail, deltas).zipped.foreach { (value, delta) => - if (delta < Byte.MaxValue) { + if (math.abs(delta) <= Byte.MaxValue) { expectResult(delta, "Wrong delta")(buffer.get()) } else { expectResult(Byte.MinValue, "Expecting escaping mark here")(buffer.get()) @@ -105,11 +106,17 @@ class IntegralDeltaSuite extends FunSuite { test(s"$scheme: simple case") { val input = columnType match { - case INT => Seq(1: Int, 2: Int, 130: Int) - case LONG => Seq(1: Long, 2: Long, 130: Long) + case INT => Seq(2: Int, 1: Int, 2: Int, 130: Int) + case LONG => Seq(2: Long, 1: Long, 2: Long, 130: Long) } skeleton(input.map(_.asInstanceOf[I#JvmType])) } + + test(s"$scheme: long random series") { + // Have to workaround with `Any` since no `ClassTag[I#JvmType]` available here. + val input = Array.fill[Any](10000)(makeRandomValue(columnType)) + skeleton(input.map(_.asInstanceOf[I#JvmType])) + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index fc053c56c052d..c36b5878cb007 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -33,6 +33,8 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -115,23 +117,31 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { case p: LogicalPlan if !p.childrenResolved => p case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => - val childOutputDataTypes = child.output.map(_.dataType) - // Only check attributes, not partitionKeys since they are always strings. - // TODO: Fully support inserting into partitioned tables. - val tableOutputDataTypes = table.attributes.map(_.dataType) - - if (childOutputDataTypes == tableOutputDataTypes) { - p - } else { - // Only do the casting when child output data types differ from table output data types. - val castedChildOutput = child.output.zip(table.output).map { - case (input, output) if input.dataType != output.dataType => - Alias(Cast(input, output.dataType), input.name)() - case (input, _) => input - } - - p.copy(child = logical.Project(castedChildOutput, child)) + castChildOutput(p, table, child) + + case p @ logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan( + _, HiveTableScan(_, table, _))), _, child, _) => + castChildOutput(p, table, child) + } + + def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) = { + val childOutputDataTypes = child.output.map(_.dataType) + // Only check attributes, not partitionKeys since they are always strings. + // TODO: Fully support inserting into partitioned tables. + val tableOutputDataTypes = table.attributes.map(_.dataType) + + if (childOutputDataTypes == tableOutputDataTypes) { + p + } else { + // Only do the casting when child output data types differ from table output data types. + val castedChildOutput = child.output.zip(table.output).map { + case (input, output) if input.dataType != output.dataType => + Alias(Cast(input, output.dataType), input.name)() + case (input, _) => input } + + p.copy(child = logical.Project(castedChildOutput, child)) + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 3ca1d93c11fa9..ac817b21a152e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. @@ -42,6 +43,9 @@ trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil + case logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan( + _, HiveTableScan(_, table, _))), partition, child, overwrite) => + InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 2fea9702954d7..465e5f146fe71 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -160,12 +160,6 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)".cmd, s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), - TestTable("dest1", - "CREATE TABLE IF NOT EXISTS dest1 (key INT, value STRING)".cmd), - TestTable("dest2", - "CREATE TABLE IF NOT EXISTS dest2 (key INT, value STRING)".cmd), - TestTable("dest3", - "CREATE TABLE IF NOT EXISTS dest3 (key INT, value STRING)".cmd), TestTable("srcpart", () => { runSqlHive( "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") @@ -257,6 +251,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { private val loadedTables = new collection.mutable.HashSet[String] + var cacheTables: Boolean = false def loadTestTable(name: String) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infite mutually recursive table loading. @@ -265,6 +260,9 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) + + if (cacheTables) + cacheTable(name) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index f9b437d435eba..55a4363af6c76 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -130,8 +130,7 @@ trait HiveFunctionFactory { } } -abstract class HiveUdf - extends Expression with Logging with HiveFunctionFactory { +abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { self: Product => type UDFType @@ -146,7 +145,7 @@ abstract class HiveUdf lazy val functionInfo = getFunctionInfo(name) lazy val function = createFunction[UDFType](name) - override def toString = s"${nodeName}#${functionInfo.getDisplayName}(${children.mkString(",")})" + override def toString = s"$nodeName#${functionInfo.getDisplayName}(${children.mkString(",")})" } case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { @@ -202,10 +201,11 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } } -case class HiveGenericUdf( - name: String, - children: Seq[Expression]) extends HiveUdf with HiveInspectors { +case class HiveGenericUdf(name: String, children: Seq[Expression]) + extends HiveUdf with HiveInspectors { + import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ + type UDFType = GenericUDF @transient @@ -357,7 +357,7 @@ case class HiveGenericUdaf( override def toString = s"$nodeName#$name(${children.mkString(",")})" - def newInstance = new HiveUdafFunction(name, children, this) + def newInstance() = new HiveUdafFunction(name, children, this) } /** @@ -435,7 +435,7 @@ case class HiveGenericUdtf( } } - override def toString() = s"$nodeName#$name(${children.mkString(",")})" + override def toString = s"$nodeName#$name(${children.mkString(",")})" } case class HiveUdafFunction( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 3cc4562a88d66..6c91f40d0f925 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -218,10 +218,7 @@ abstract class HiveComparisonTest val quotes = "\"\"\"" queryList.zipWithIndex.map { case (query, i) => - s""" - |val q$i = $quotes$query$quotes.q - |q$i.stringResult() - """.stripMargin + s"""val q$i = hql($quotes$query$quotes); q$i.collect()""" }.mkString("\n== Console version of this test ==\n", "\n", "\n") } @@ -287,7 +284,6 @@ abstract class HiveComparisonTest |Error: ${e.getMessage} |${stackTraceToString(e)} |$queryString - |$consoleTestCase """.stripMargin stringToFile( new File(hiveFailedDirectory, testCaseName), @@ -304,7 +300,7 @@ abstract class HiveComparisonTest val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => val query = new TestHive.HiveQLQueryExecution(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { - case e: Exception => + case e: Throwable => val errorMessage = s""" |Failed to execute query using catalyst: @@ -313,8 +309,6 @@ abstract class HiveComparisonTest |$query |== HIVE - ${hive.size} row(s) == |${hive.mkString("\n")} - | - |$consoleTestCase """.stripMargin stringToFile(new File(failedDirectory, testCaseName), errorMessage + consoleTestCase) fail(errorMessage) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index f76e16bc1afc5..c3cfa3d25a5c2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -17,16 +17,26 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfter + import org.apache.spark.sql.hive.TestHive /** * Runs the test cases that are included in the hive distribution. */ -class HiveCompatibilitySuite extends HiveQueryFileTest { +class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // TODO: bundle in jar files... get from classpath lazy val hiveQueryDir = TestHive.getHiveFile("ql/src/test/queries/clientpositive") def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) + override def beforeAll() { + TestHive.cacheTables = true + } + + override def afterAll() { + TestHive.cacheTables = false + } + /** A list of tests deemed out of scope currently and thus completely disregarded. */ override def blackList = Seq( // These tests use hooks that are not on the classpath and thus break all subsequent execution. From 268b53567c93538c03cb66276ed9e05c9f1d3ac6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 14 Apr 2014 15:51:54 -0700 Subject: [PATCH 27/61] HOTFIX: Use file name and not paths for excludes --- .rat-excludes | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.rat-excludes b/.rat-excludes index 8954330bd10a7..9e9abb3f10bbf 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -40,5 +40,5 @@ work golden test.out/* .*iml -python/metastore/service.properties -python/metastore/db.lck +service.properties +db.lck From 0247b5c5467ca1b0d03ba929a78fa4d805582d84 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 14 Apr 2014 19:50:00 -0700 Subject: [PATCH 28/61] SPARK-1488. Resolve scalac feature warnings during build For your consideration: scalac currently notes a number of feature warnings during compilation: ``` [warn] there were 65 feature warning(s); re-run with -feature for details ``` Warnings are like: ``` [warn] /Users/srowen/Documents/spark/core/src/main/scala/org/apache/spark/SparkContext.scala:1261: implicit conversion method rddToPairRDDFunctions should be enabled [warn] by making the implicit value scala.language.implicitConversions visible. [warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions' [warn] or by setting the compiler option -language:implicitConversions. [warn] See the Scala docs for value scala.language.implicitConversions for a discussion [warn] why the feature should be explicitly enabled. [warn] implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = [warn] ^ ``` scalac is suggesting that it's just best practice to explicitly enable certain language features by importing them where used. This PR simply adds the imports it suggests (and squashes one other Java warning along the way). This leaves just deprecation warnings in the build. Author: Sean Owen Closes #404 from srowen/SPARK-1488 and squashes the following commits: 8598980 [Sean Owen] Quiet scalac warnings about language features by explicitly importing language features. 39bc831 [Sean Owen] Enable -feature in scalac to emit language feature warnings --- bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala | 2 ++ core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + .../main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala | 1 + .../src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 1 + core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala | 1 + .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 1 + .../scala/org/apache/spark/deploy/FaultToleranceTest.scala | 1 + .../src/main/scala/org/apache/spark/deploy/master/Master.scala | 1 + .../src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 1 + .../scala/org/apache/spark/network/ConnectionManager.scala | 1 + core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 1 + core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala | 1 + .../scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala | 1 + .../src/main/scala/org/apache/spark/scheduler/ResultTask.scala | 1 + .../main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 1 + .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 1 + core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 1 + .../org/apache/spark/util/TimeStampedWeakValueHashMap.scala | 1 + core/src/main/scala/org/apache/spark/util/Vector.scala | 1 + .../test/scala/org/apache/spark/ConnectionManagerSuite.scala | 2 +- core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala | 2 ++ core/src/test/scala/org/apache/spark/DriverSuite.scala | 2 ++ core/src/test/scala/org/apache/spark/PipedRDDSuite.scala | 3 +-- .../test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala | 1 + .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 1 + .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 3 +++ .../scala/org/apache/spark/storage/DiskBlockManagerSuite.scala | 1 + core/src/test/scala/org/apache/spark/ui/UISuite.scala | 1 + .../org/apache/spark/util/random/XORShiftRandomSuite.scala | 2 ++ .../org/apache/spark/streaming/examples/ZeroMQWordCount.scala | 2 ++ graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 1 + .../org/apache/spark/graphx/impl/MessageToPartition.scala | 1 + .../main/scala/org/apache/spark/graphx/impl/Serializers.scala | 3 ++- .../scala/org/apache/spark/graphx/util/BytecodeUtils.scala | 1 + .../main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala | 1 + .../scala/org/apache/spark/mllib/util/MFDataGenerator.scala | 1 + .../java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java | 1 + pom.xml | 1 + project/SparkBuild.scala | 2 +- .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 1 + .../scala/org/apache/spark/streaming/StreamingContext.scala | 1 + .../org/apache/spark/streaming/api/java/JavaDStream.scala | 1 + .../org/apache/spark/streaming/api/java/JavaDStreamLike.scala | 1 + .../org/apache/spark/streaming/api/java/JavaPairDStream.scala | 1 + .../org/apache/spark/streaming/receivers/ActorReceiver.scala | 1 + .../apache/spark/streaming/scheduler/NetworkInputTracker.scala | 1 + .../org/apache/spark/streaming/StreamingContextSuite.scala | 2 ++ .../src/test/scala/org/apache/spark/streaming/UISuite.scala | 1 + .../org/apache/spark/tools/JavaAPICompletenessChecker.scala | 1 + 49 files changed, 58 insertions(+), 5 deletions(-) diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 69144e3e657bf..8e0f82ddb8897 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -24,6 +24,8 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.storage.StorageLevel +import scala.language.postfixOps + class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable class TestMessage(val targetId: String) extends Message[String] with Serializable diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 456070fa7c5ef..3ddc0d5eeefb8 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -25,6 +25,7 @@ import java.util.UUID.randomUUID import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.language.implicitConversions import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 537f410b0ca26..4330cef3965ee 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -19,6 +19,7 @@ package org.apache.spark.api.java import java.lang.{Double => JDouble} +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.Partitioner diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index a41c7dbda2afc..e5b2c8a5e7cb1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -21,6 +21,7 @@ import java.util.{Comparator, List => JList} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 01d9357a2556d..327c1552dc941 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -17,6 +17,7 @@ package org.apache.spark.api.java +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 7fbefe1cb0fb1..e6a3f06b0ea42 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -22,6 +22,7 @@ import java.util.{Map => JMap} import scala.collection.JavaConversions import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index f4eb1601be3e4..47dbcd87c35b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ListBuffer import scala.concurrent.{Await, future, promise} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ +import scala.language.postfixOps import scala.sys.process._ import org.json4s._ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 6c58e741df001..81f990bfa6513 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -23,6 +23,7 @@ import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.postfixOps import scala.util.Random import akka.actor._ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 52c164ca3c574..dd0a1360abe14 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -23,6 +23,7 @@ import java.util.Date import scala.collection.mutable.HashMap import scala.concurrent.duration._ +import scala.language.postfixOps import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index cfee41c61362e..dcbbc1853186b 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -33,6 +33,7 @@ import scala.collection.mutable.SynchronizedQueue import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.concurrent.duration._ +import scala.language.postfixOps import org.apache.spark._ import org.apache.spark.util.{SystemClock, Utils} diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index c6e79557f08a1..9ff76892aed32 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 44401a663440c..c45b759f007cc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -21,6 +21,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 7367c08b5d324..0800c5684c60f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.util.Properties import scala.collection.mutable.Map +import scala.language.existentials import org.apache.spark._ import org.apache.spark.executor.TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 0b381308b61ff..0e8d551e4b2ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -21,6 +21,7 @@ import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap +import scala.language.existentials import org.apache.spark._ import org.apache.spark.rdd.{RDD, RDDCheckpointData} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 23f3b3e824762..02b62de7e36b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -21,6 +21,7 @@ import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap +import scala.language.existentials import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index acd152dda89d4..a3439b525fde1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -25,6 +25,7 @@ import scala.concurrent.duration._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +import scala.language.postfixOps import scala.util.Random import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 3ae147a36c8a4..750f5a501c213 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -22,6 +22,7 @@ import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.annotation.tailrec +import scala.language.implicitConversions import scala.util.{Failure, Success, Try} import scala.xml.Node diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala index b65017d6806c6..f5be5856c2109 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala @@ -21,6 +21,7 @@ import java.lang.ref.WeakReference import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable +import scala.language.implicitConversions import org.apache.spark.Logging diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index 1a647fa1c9d84..c6cab82c3e546 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -17,6 +17,7 @@ package org.apache.spark.util +import scala.language.implicitConversions import scala.util.Random import org.apache.spark.util.random.XORShiftRandom diff --git a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala index 80f7ec00c74b2..df6b2604c8d8a 100644 --- a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.network.{ConnectionManager, Message, ConnectionManagerId import scala.concurrent.Await import scala.concurrent.TimeoutException import scala.concurrent.duration._ - +import scala.language.postfixOps /** * Test the ConnectionManager with various security settings. diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index e50981cf6fb20..5a8310090890d 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark import java.lang.ref.WeakReference import scala.collection.mutable.{HashSet, SynchronizedSet} +import scala.language.existentials +import scala.language.postfixOps import scala.util.Random import org.scalatest.{BeforeAndAfter, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 7f59bdcce4cc7..de4bd90c8f7e5 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -29,6 +29,8 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.Utils +import scala.language.postfixOps + class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 867b28cc0d971..dfe057515efa0 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark import java.io.File -import com.google.common.io.Files - import org.scalatest.FunSuite import org.apache.spark.rdd.{HadoopRDD, PipedRDD, HadoopPartition} @@ -28,6 +26,7 @@ import org.apache.hadoop.mapred.{JobConf, TextInputFormat, FileSplit} import org.apache.hadoop.fs.Path import scala.collection.Map +import scala.language.postfixOps import scala.sys.process._ import scala.util.Try diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 3b833f2e41867..28197657e9bad 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -22,6 +22,7 @@ import java.util.concurrent.Semaphore import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global +import scala.language.postfixOps import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Timeouts diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index db4df1d1212ff..35a7ac9d049c2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import scala.Tuple2 import scala.collection.mutable.{HashSet, HashMap, Map} +import scala.language.reflectiveCalls import org.scalatest.{BeforeAndAfter, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index e10ec7d2624a0..907428db80af3 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -33,6 +33,9 @@ import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import scala.language.implicitConversions +import scala.language.postfixOps + class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { private val conf = new SparkConf(false) var store: BlockManager = null diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 808ddfdcf45d8..9b29e2a8a55de 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.io.{File, FileWriter} import scala.collection.mutable +import scala.language.reflectiveCalls import com.google.common.io.Files import org.scalatest.{BeforeAndAfterEach, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index b85c483ca2a08..ed02b0ba00d43 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -21,6 +21,7 @@ import java.net.ServerSocket import javax.servlet.http.HttpServletRequest import scala.io.Source +import scala.language.postfixOps import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 39199a1a17ccd..0865c6386f7cd 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -22,6 +22,8 @@ import org.scalatest.matchers.ShouldMatchers import org.apache.spark.util.Utils.times +import scala.language.reflectiveCalls + class XORShiftRandomSuite extends FunSuite with ShouldMatchers { def fixture = new { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 445d2028582af..6f88db1abf19d 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -27,6 +27,8 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.zeromq._ +import scala.language.implicitConversions + /** * A simple publisher for demonstration purposes, repeatedly publishes random Messages * every one second. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index ef05623d7a0a1..45349692cbf6c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -17,6 +17,7 @@ package org.apache.spark.graphx +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.graphx.impl._ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index 9d4f3750cb8e4..c45ba3d2f8c24 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -17,6 +17,7 @@ package org.apache.spark.graphx.impl +import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import org.apache.spark.Partitioner diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 2f2c524df6394..2f0531ee5f379 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -20,10 +20,11 @@ package org.apache.spark.graphx.impl import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import org.apache.spark.SparkConf import org.apache.spark.graphx._ import org.apache.spark.serializer._ +import scala.language.existentials + private[graphx] class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index 087b1156f690b..be6b9047d932d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.HashSet +import scala.language.existentials import org.apache.spark.util.Utils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala index 873de871fd884..365b5e75d7f75 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.rdd +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 3f413faca6bb4..b76fbe89c3681 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.util +import scala.language.postfixOps import scala.util.Random import org.jblas.DoubleMatrix diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java index c6d8425ffc38d..1421067dc61ed 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -36,6 +36,7 @@ public void denseArrayConstruction() { @Test public void sparseArrayConstruction() { + @SuppressWarnings("unchecked") Vector v = Vectors.sparse(3, Lists.>newArrayList( new Tuple2(0, 2.0), new Tuple2(2, 3.0))); diff --git a/pom.xml b/pom.xml index 5f66cbe768592..0eacedf7a6533 100644 --- a/pom.xml +++ b/pom.xml @@ -648,6 +648,7 @@ -unchecked -deprecation + -feature -Xms1024m diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a6058bba3d211..aac07b9f6129d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -162,7 +162,7 @@ object SparkBuild extends Build { organization := "org.apache.spark", version := SPARK_VERSION, scalaVersion := "2.10.4", - scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", + scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-feature", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 5b6aea81cb7d1..13a19d0adf5e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst +import scala.language.implicitConversions import scala.util.parsing.combinator.lexical.StdLexical import scala.util.parsing.combinator.syntactical.StandardTokenParsers import scala.util.parsing.input.CharArrayReader.EofCh diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index ff5d0aaa3d0bd..e9a4f7ba22576 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.Map import scala.collection.mutable.Queue +import scala.language.implicitConversions import scala.reflect.ClassTag import akka.actor.{Props, SupervisorStrategy} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 721d50273259e..13e2bacc92edc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -23,6 +23,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.storage.StorageLevel import org.apache.spark.rdd.RDD +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.DStream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index bb2f492d06a00..a6184de4e83c1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -22,6 +22,7 @@ import java.lang.{Long => JLong} import java.util.{List => JList} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaRDDLike} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 2ac943d7bf781..cb8e8f00a7b82 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -21,6 +21,7 @@ import java.lang.{Long => JLong, Iterable => JIterable} import java.util.{List => JList} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index f5984d03c5342..da0d364ae7bdb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -23,6 +23,7 @@ import akka.actor.{ PossiblyHarmful, OneForOneStrategy } import akka.actor.SupervisorStrategy._ import scala.concurrent.duration._ +import scala.language.postfixOps import scala.reflect.ClassTag import org.apache.spark.storage.{StorageLevel, StreamBlockId} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index a1e6f5176825a..438e72a7ced89 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} +import scala.language.existentials import akka.actor._ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index efd0d22ecb57a..ad5367ab941a2 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -28,6 +28,8 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ +import scala.language.postfixOps + class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { val master = "local[2]" diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 35538ec188f67..031e93ab24a70 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming import scala.io.Source +import scala.language.postfixOps import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala index 420522433e1e8..8d0f09933c8d3 100644 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -20,6 +20,7 @@ package org.apache.spark.tools import java.lang.reflect.Method import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import org.apache.spark._ import org.apache.spark.api.java._ From c99bcb7feaa761c5826f2e1d844d0502a3b79538 Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Tue, 15 Apr 2014 00:07:55 -0700 Subject: [PATCH 29/61] SPARK-1374: PySpark API for SparkSQL An initial API that exposes SparkSQL functionality in PySpark. A PythonRDD composed of dictionaries, with string keys and primitive values (boolean, float, int, long, string) can be converted into a SchemaRDD that supports sql queries. ``` from pyspark.context import SQLContext sqlCtx = SQLContext(sc) rdd = sc.parallelize([{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}]) srdd = sqlCtx.applySchema(rdd) sqlCtx.registerRDDAsTable(srdd, "table1") srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") srdd2.collect() ``` The last line yields ```[{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, {"f1" : 3, "f2": "row3"}]``` Author: Ahir Reddy Author: Michael Armbrust Closes #363 from ahirreddy/pysql and squashes the following commits: 0294497 [Ahir Reddy] Updated log4j properties to supress Hive Warns 307d6e0 [Ahir Reddy] Style fix 6f7b8f6 [Ahir Reddy] Temporary fix MIMA checker. Since we now assemble Spark jar with Hive, we don't want to check the interfaces of all of our hive dependencies 3ef074a [Ahir Reddy] Updated documentation because classes moved to sql.py 29245bf [Ahir Reddy] Cache underlying SchemaRDD instead of generating and caching PythonRDD f2312c7 [Ahir Reddy] Moved everything into sql.py a19afe4 [Ahir Reddy] Doc fixes 6d658ba [Ahir Reddy] Remove the metastore directory created by the HiveContext tests in SparkSQL 521ff6d [Ahir Reddy] Trying to get spark to build with hive ab95eba [Ahir Reddy] Set SPARK_HIVE=true on jenkins ded03e7 [Ahir Reddy] Added doc test for HiveContext 22de1d4 [Ahir Reddy] Fixed maven pyrolite dependency e4da06c [Ahir Reddy] Display message if hive is not built into spark 227a0be [Michael Armbrust] Update API links. Fix Hive example. 58e2aa9 [Michael Armbrust] Build Docs for pyspark SQL Api. Minor fixes. 4285340 [Michael Armbrust] Fix building of Hive API Docs. 38a92b0 [Michael Armbrust] Add note to future non-python developers about python docs. 337b201 [Ahir Reddy] Changed com.clearspring.analytics stream version from 2.4.0 to 2.5.1 to match SBT build, and added pyrolite to maven build 40491c9 [Ahir Reddy] PR Changes + Method Visibility 1836944 [Michael Armbrust] Fix comments. e00980f [Michael Armbrust] First draft of python sql programming guide. b0192d3 [Ahir Reddy] Added Long, Double and Boolean as usable types + unit test f98a422 [Ahir Reddy] HiveContexts 79621cf [Ahir Reddy] cleaning up cruft b406ba0 [Ahir Reddy] doctest formatting 20936a5 [Ahir Reddy] Added tests and documentation e4d21b4 [Ahir Reddy] Added pyrolite dependency 79f739d [Ahir Reddy] added more tests 7515ba0 [Ahir Reddy] added more tests :) d26ec5e [Ahir Reddy] added test e9f5b8d [Ahir Reddy] adding tests 906d180 [Ahir Reddy] added todo explaining cost of creating Row object in python 251f99d [Ahir Reddy] for now only allow dictionaries as input 09b9980 [Ahir Reddy] made jrdd explicitly lazy c608947 [Ahir Reddy] SchemaRDD now has all RDD operations 725c91e [Ahir Reddy] awesome row objects 55d1c76 [Ahir Reddy] return row objects 4fe1319 [Ahir Reddy] output dictionaries correctly be079de [Ahir Reddy] returning dictionaries works cd5f79f [Ahir Reddy] Switched to using Scala SQLContext e948bd9 [Ahir Reddy] yippie 4886052 [Ahir Reddy] even better c0fb1c6 [Ahir Reddy] more working 043ca85 [Ahir Reddy] working 5496f9f [Ahir Reddy] doesn't crash b8b904b [Ahir Reddy] Added schema rdd class 67ba875 [Ahir Reddy] java to python, and python to java bcc0f23 [Ahir Reddy] Java to python ab6025d [Ahir Reddy] compiling --- core/pom.xml | 5 + .../apache/spark/api/python/PythonRDD.scala | 32 ++ dev/run-tests | 1 + docs/README.md | 2 +- docs/_plugins/copy_api_dirs.rb | 4 +- docs/sql-programming-guide.md | 103 ++++- pom.xml | 2 +- project/SparkBuild.scala | 3 +- python/pyspark/__init__.py | 18 +- python/pyspark/java_gateway.py | 4 + python/pyspark/sql.py | 363 ++++++++++++++++++ python/run-tests | 4 + .../org/apache/spark/sql/SQLContext.scala | 27 ++ .../org/apache/spark/sql/SchemaRDD.scala | 23 ++ .../org/apache/spark/sql/hive/TestHive.scala | 3 +- sql/hive/src/test/resources/log4j.properties | 3 + .../spark/tools/GenerateMIMAIgnore.scala | 4 +- 17 files changed, 589 insertions(+), 12 deletions(-) create mode 100644 python/pyspark/sql.py diff --git a/core/pom.xml b/core/pom.xml index a1bdd8ec68aeb..d87e2bca030e3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -266,6 +266,11 @@ junit-interface test + + org.spark-project + pyrolite + 2.0 + target/scala-${scala.binary.version}/classes diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 32f1100406d74..f9d86fed34d0f 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -25,6 +25,8 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import net.razorvine.pickle.{Pickler, Unpickler} + import org.apache.spark._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast @@ -284,6 +286,36 @@ private[spark] object PythonRDD { file.close() } + /** + * Convert an RDD of serialized Python dictionaries to Scala Maps + * TODO: Support more Python types. + */ + def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { + pyRDD.rdd.mapPartitions { iter => + val unpickle = new Unpickler + // TODO: Figure out why flatMap is necessay for pyspark + iter.flatMap { row => + unpickle.loads(row) match { + case objs: java.util.ArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) + // Incase the partition doesn't have a collection + case obj: JMap[String @unchecked, _] => Seq(obj.toMap) + } + } + } + } + + /** + * Convert and RDD of Java objects to and RDD of serialized Python objects, that is usable by + * PySpark. + */ + def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { + jRDD.rdd.mapPartitions { iter => + val pickle = new Pickler + iter.map { row => + pickle.dumps(row) + } + } + } } private diff --git a/dev/run-tests b/dev/run-tests index 6ad674a2ba127..0725b681f1a1b 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -34,6 +34,7 @@ else fi JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') [ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run because JDK version is < 1.8." +export SPARK_HIVE=true echo "=========================================================================" echo "Running Apache RAT checks" diff --git a/docs/README.md b/docs/README.md index 0678fc5c86706..75b1811ba99af 100644 --- a/docs/README.md +++ b/docs/README.md @@ -42,7 +42,7 @@ To mark a block of code in your markdown to be syntax highlighted by jekyll duri You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. -Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. +Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. Documentation is only generated for classes that are listed as public in `__init__.py`. When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index bbd56d2fd13bb..05f0bd47a88a5 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -32,8 +32,8 @@ curr_dir = pwd cd("..") - puts "Running sbt/sbt doc from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt doc` + puts "Running 'sbt/sbt doc hive/doc' from " + pwd + "; this may take a few minutes..." + puts `sbt/sbt doc hive/doc` puts "Moving back into docs dir." cd("docs") diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a59393e1424de..6f616fb7c2448 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -20,7 +20,7 @@ a schema that describes the data types of each column in the row. A SchemaRDD i in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). -**All of the examples on this page use sample data included in the Spark distribution and can be run in the spark-shell.** +**All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`.** @@ -33,6 +33,19 @@ a schema that describes the data types of each column in the row. A JavaSchemaR in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). + +
      + +Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using +Spark. At the core of this component is a new type of RDD, +[SchemaRDD](api/pyspark/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed +[Row](api/pyspark/pyspark.sql.Row-class.html) objects along with +a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table +in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet +file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). + +**All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell.** +
      *************************************************************************************************** @@ -44,7 +57,7 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac The entry point into all relational functionality in Spark is the [SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its -decendents. To create a basic SQLContext, all you need is a SparkContext. +descendants. To create a basic SQLContext, all you need is a SparkContext. {% highlight scala %} val sc: SparkContext // An existing SparkContext. @@ -60,7 +73,7 @@ import sqlContext._ The entry point into all relational functionality in Spark is the [JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one -of its decendents. To create a basic JavaSQLContext, all you need is a JavaSparkContext. +of its descendants. To create a basic JavaSQLContext, all you need is a JavaSparkContext. {% highlight java %} JavaSparkContext ctx = ...; // An existing JavaSparkContext. @@ -69,6 +82,19 @@ JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx); +
      + +The entry point into all relational functionality in Spark is the +[SQLContext](api/pyspark/pyspark.sql.SQLContext-class.html) class, or one +of its decedents. To create a basic SQLContext, all you need is a SparkContext. + +{% highlight python %} +from pyspark.sql import SQLContext +sqlCtx = SQLContext(sc) +{% endhighlight %} + +
      + ## Running SQL on RDDs @@ -81,7 +107,7 @@ One type of table that is supported by Spark SQL is an RDD of Scala case classes defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be -registered as a table. Tables can used in subsequent SQL statements. +registered as a table. Tables can be used in subsequent SQL statements. {% highlight scala %} val sqlContext = new org.apache.spark.sql.SQLContext(sc) @@ -176,6 +202,34 @@ List teenagerNames = teenagers.map(new Function() { +
      + +One type of table that is supported by Spark SQL is an RDD of dictionaries. The keys of the +dictionary define the columns names of the table, and the types are inferred by looking at the first +row. Any RDD of dictionaries can converted to a SchemaRDD and then registered as a table. Tables +can be used in subsequent SQL statements. + +{% highlight python %} +# Load a text file and convert each line to a dictionary. +lines = sc.textFile("examples/src/main/resources/people.txt") +parts = lines.map(lambda l: l.split(",")) +people = parts.map(lambda p: {"name": p[0], "age": int(p[1])}) + +# Infer the schema, and register the SchemaRDD as a table. +# In future versions of PySpark we would like to add support for registering RDDs with other +# datatypes as tables +peopleTable = sqlCtx.inferSchema(people) +peopleTable.registerAsTable("people") + +# SQL can be run over SchemaRDDs that have been registered as a table. +teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + +# The results of SQL queries are RDDs and support all the normal RDD operations. +teenNames = teenagers.map(lambda p: "Name: " + p.name) +{% endhighlight %} + +
      + **Note that Spark SQL currently uses a very basic SQL parser.** @@ -231,6 +285,27 @@ parquetFile.registerAsTable("parquetFile"); JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); +{% endhighlight %} + + + +
      + +{% highlight python %} + +peopleTable # The SchemaRDD from the previous example. + +# SchemaRDDs can be saved as parquet files, maintaining the schema information. +peopleTable.saveAsParquetFile("people.parquet") + +# Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. +# The result of loading a parquet file is also a SchemaRDD. +parquetFile = sqlCtx.parquetFile("people.parquet") + +# Parquet files can also be registered as tables and then used in SQL statements. +parquetFile.registerAsTable("parquetFile"); +teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") + {% endhighlight %}
      @@ -318,4 +393,24 @@ Row[] results = hiveCtx.hql("FROM src SELECT key, value").collect(); +
      + +When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and +adds support for finding tables in in the MetaStore and writing queries using HiveQL. In addition to +the `sql` method a `HiveContext` also provides an `hql` methods, which allows queries to be +expressed in HiveQL. + +{% highlight python %} + +from pyspark.sql import HiveContext +hiveCtx = HiveContext(sc) + +hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + +# Queries can be expressed in HiveQL. +results = hiveCtx.hql("FROM src SELECT key, value").collect() + +{% endhighlight %} +
      diff --git a/pom.xml b/pom.xml index 0eacedf7a6533..cd204376de5db 100644 --- a/pom.xml +++ b/pom.xml @@ -262,7 +262,7 @@ com.clearspring.analytics stream - 2.4.0 + 2.5.1