diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 44e26bbb9e094c37527122bb3825197436c13219..281216612fc191f7945bdb5094ffb381d2e31a38 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -28,21 +28,22 @@ object Bagel extends Logging { /** * Runs a Bagel program. * @param sc [[org.apache.spark.SparkContext]] to use for the program. - * @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the Key will be - * the vertex id. - * @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often this will be an - * empty array, i.e. sc.parallelize(Array[K, Message]()). - * @param combiner [[org.apache.spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one - * message before sending (which often involves network I/O). - * @param aggregator [[org.apache.spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep, - * and provides the result to each vertex in the next superstep. + * @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the + * Key will be the vertex id. + * @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often + * this will be an empty array, i.e. sc.parallelize(Array[K, Message]()). + * @param combiner [[org.apache.spark.bagel.Combiner]] combines multiple individual messages to a + * given vertex into one message before sending (which often involves network I/O). + * @param aggregator [[org.apache.spark.bagel.Aggregator]] performs a reduce across all vertices + * after each superstep and provides the result to each vertex in the next + * superstep. * @param partitioner [[org.apache.spark.Partitioner]] partitions values by key * @param numPartitions number of partitions across which to split the graph. * Default is the default parallelism of the SparkContext - * @param storageLevel [[org.apache.spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep. - * Defaults to caching in memory. - * @param compute function that takes a Vertex, optional set of (possibly combined) messages to the Vertex, - * optional Aggregator and the current superstep, + * @param storageLevel [[org.apache.spark.storage.StorageLevel]] to use for caching of + * intermediate RDDs in each superstep. Defaults to caching in memory. + * @param compute function that takes a Vertex, optional set of (possibly combined) messages to + * the Vertex, optional Aggregator and the current superstep, * and returns a set of (Vertex, outgoing Messages) pairs * @tparam K key * @tparam V vertex type @@ -71,7 +72,7 @@ object Bagel extends Logging { var msgs = messages var noActivity = false do { - logInfo("Starting superstep "+superstep+".") + logInfo("Starting superstep " + superstep + ".") val startTime = System.currentTimeMillis val aggregated = agg(verts, aggregator) @@ -97,7 +98,8 @@ object Bagel extends Logging { verts } - /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default storage level */ + /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default + * storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], @@ -106,8 +108,8 @@ object Bagel extends Logging { partitioner: Partitioner, numPartitions: Int )( - compute: (V, Option[C], Int) => (V, Array[M]) - ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) + compute: (V, Option[C], Int) => (V, Array[M])): RDD[(K, V)] = run(sc, vertices, messages, + combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( @@ -127,8 +129,8 @@ object Bagel extends Logging { } /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]] - * and default storage level + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default + * [[org.apache.spark.HashPartitioner]] and default storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, @@ -138,9 +140,13 @@ object Bagel extends Logging { numPartitions: Int )( compute: (V, Option[C], Int) => (V, Array[M]) - ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) + ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, + DEFAULT_STORAGE_LEVEL)(compute) - /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default [[org.apache.spark.HashPartitioner]]*/ + /** + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the + * default [[org.apache.spark.HashPartitioner]] + */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], @@ -158,7 +164,8 @@ object Bagel extends Logging { } /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]], + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], + * default [[org.apache.spark.HashPartitioner]], * [[org.apache.spark.bagel.DefaultCombiner]] and the default storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( @@ -171,7 +178,8 @@ object Bagel extends Logging { ): RDD[(K, V)] = run(sc, vertices, messages, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) /** - * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], the default [[org.apache.spark.HashPartitioner]] + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], + * the default [[org.apache.spark.HashPartitioner]] * and [[org.apache.spark.bagel.DefaultCombiner]] */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( @@ -227,8 +235,9 @@ object Bagel extends Logging { }) numMsgs += newMsgs.size - if (newVert.active) + if (newVert.active) { numActiveVerts += 1 + } Some((newVert, newMsgs)) }.persist(storageLevel) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 8e5dd8a85020d4d3eb9bdf259c652638df6834a9..15a0d24fd954eaa9d5acde7adbc0aae8210ed680 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -31,8 +31,8 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { private val loading = new HashSet[RDDBlockId]() /** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */ - def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, storageLevel: StorageLevel) - : Iterator[T] = { + def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, + storageLevel: StorageLevel): Iterator[T] = { val key = RDDBlockId(rdd.id, split.index) logDebug("Looking for partition " + key) blockManager.get(key) match { diff --git a/core/src/main/scala/org/apache/spark/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/FetchFailedException.scala index d242047502fd3acb320264533ef88a4443c4fe3b..8eaa26bdb1b5b866d507213672f5c0219351b48f 100644 --- a/core/src/main/scala/org/apache/spark/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/FetchFailedException.scala @@ -25,7 +25,8 @@ private[spark] class FetchFailedException( cause: Throwable) extends Exception { - def this (bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int, cause: Throwable) = + def this (bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int, + cause: Throwable) = this(FetchFailed(bmAddress, shuffleId, mapId, reduceId), "Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId), cause) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 30d182b008930af670b7e9120e78ed69189febd3..8d6db0fca23f7e811a8b42bc6d327c3402331c8f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -29,7 +29,7 @@ import akka.pattern.ask import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 566472e5979585d349547fdc3b2ad06ec09a442c..25f7a5ed1c250f22f033be3bbc2d2178a53cbc8d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -63,9 +63,9 @@ import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, Metada */ class SparkContext( config: SparkConf, - // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, etc) - // too. This is typically generated from InputFormatInfo.computePreferredLocations. It contains - // a map from hostname to a list of input format splits on the host. + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, + // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It + // contains a map from hostname to a list of input format splits on the host. val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) extends Logging { @@ -552,10 +552,11 @@ class SparkContext( /** * Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and - * BytesWritable values that contain a serialized partition. This is still an experimental storage - * format and may not be supported exactly as is in future Spark releases. It will also be pretty - * slow if you use the default serializer (Java serialization), though the nice thing about it is - * that there's very little effort required to save arbitrary objects. + * BytesWritable values that contain a serialized partition. This is still an experimental + * storage format and may not be supported exactly as is in future Spark releases. It will also + * be pretty slow if you use the default serializer (Java serialization), + * though the nice thing about it is that there's very little effort required to save arbitrary + * objects. */ def objectFile[T: ClassTag]( path: String, @@ -1043,7 +1044,7 @@ object SparkContext { implicit object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long) = t1 + t2 - def zero(initialValue: Long) = 0l + def zero(initialValue: Long) = 0L } implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { @@ -1109,7 +1110,8 @@ object SparkContext { implicit def floatWritableConverter() = simpleWritableConverter[Float, FloatWritable](_.get) - implicit def booleanWritableConverter() = simpleWritableConverter[Boolean, BooleanWritable](_.get) + implicit def booleanWritableConverter() = + simpleWritableConverter[Boolean, BooleanWritable](_.get) implicit def bytesWritableConverter() = { simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes) @@ -1258,7 +1260,8 @@ object SparkContext { case "yarn-client" => val scheduler = try { - val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") + val clazz = + Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] @@ -1269,7 +1272,8 @@ object SparkContext { } val backend = try { - val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") + val clazz = + Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index ed788560e79f17c2e22c6a9527f4b9b29513124a..6ae020f6a21b1da3f21881e18f30e27cd808b7d0 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -96,7 +96,7 @@ object SparkEnv extends Logging { @volatile private var lastSetSparkEnv : SparkEnv = _ def set(e: SparkEnv) { - lastSetSparkEnv = e + lastSetSparkEnv = e env.set(e) } @@ -112,7 +112,7 @@ object SparkEnv extends Logging { * Returns the ThreadLocal SparkEnv. */ def getThreadLocal: SparkEnv = { - env.get() + env.get() } private[spark] def create( @@ -168,7 +168,8 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf)), conf) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, + serializer, conf) val connectionManager = blockManager.connectionManager 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 b0dedc6f4eb135f5751414921d21eb4fd2110080..33737e1960aded65caf9494a8eda115e376f1ec1 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 @@ -148,8 +148,8 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav def sum(): Double = srdd.sum() /** - * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and count - * of the RDD's elements in one operation. + * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and + * count of the RDD's elements in one operation. */ def stats(): StatCounter = srdd.stats() 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 f430a33db1e4a3b79b9352d2756842d62d408e2d..5b1bf9476e4d5a4076a2f74453c500b48c03f398 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 @@ -88,7 +88,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numPartitions: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct(numPartitions)) + def distinct(numPartitions: Int): JavaPairRDD[K, V] = + new JavaPairRDD[K, V](rdd.distinct(numPartitions)) /** * Return a new RDD containing only the elements that satisfy a predicate. @@ -210,25 +211,25 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). + * Merge the values for each key using an associative function and a neutral "zero value" which + * may be added to the result an arbitrary number of times, and must not change the result + * (e.g ., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ - def foldByKey(zeroValue: V, partitioner: Partitioner, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = - fromRDD(rdd.foldByKey(zeroValue, partitioner)(func)) + def foldByKey(zeroValue: V, partitioner: Partitioner, func: JFunction2[V, V, V]) + : JavaPairRDD[K, V] = fromRDD(rdd.foldByKey(zeroValue, partitioner)(func)) /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). + * Merge the values for each key using an associative function and a neutral "zero value" which + * may be added to the result an arbitrary number of times, and must not change the result + * (e.g ., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, numPartitions: Int, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = fromRDD(rdd.foldByKey(zeroValue, numPartitions)(func)) /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). + * Merge the values for each key using an associative function and a neutral "zero value" + * which may be added to the result an arbitrary number of times, and must not change the result + * (e.g., Nil for list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = fromRDD(rdd.foldByKey(zeroValue)(func)) @@ -375,7 +376,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * into `numPartitions` partitions. */ - def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Optional[W])] = { + def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int) + : JavaPairRDD[K, (V, Optional[W])] = { val joinResult = rdd.leftOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -397,7 +399,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Optional[V], W)] = { + def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int) + : JavaPairRDD[K, (Optional[V], W)] = { val joinResult = rdd.rightOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -439,8 +442,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], + partitioner: Partitioner): JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) /** @@ -462,8 +465,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] - = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) + def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int) + : JavaPairRDD[K, (JList[V], JList[W])] = + fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 4db7339e6716bc25e3035257f80854d30b9fc46e..fcb9729c10a6d17e3eda3f45fd5bd218e7a85c65 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -76,7 +76,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), - preservesPartitioning)) + preservesPartitioning)) /** * Return a new RDD by applying a function to all elements of this RDD. @@ -134,7 +134,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U], preservesPartitioning: Boolean): JavaRDD[U] = { + def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U], + preservesPartitioning: Boolean): JavaRDD[U] = { def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) JavaRDD.fromRDD(rdd.mapPartitions(fn, preservesPartitioning)(f.elementType()))(f.elementType()) } @@ -160,16 +161,18 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions(f: DoubleFlatMapFunction[java.util.Iterator[T]], preservesPartitioning: Boolean): JavaDoubleRDD = { + def mapPartitions(f: DoubleFlatMapFunction[java.util.Iterator[T]], + preservesPartitioning: Boolean): JavaDoubleRDD = { def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) - new JavaDoubleRDD(rdd.mapPartitions(fn, preservesPartitioning).map((x: java.lang.Double) => x.doubleValue())) + new JavaDoubleRDD(rdd.mapPartitions(fn, preservesPartitioning) + .map((x: java.lang.Double) => x.doubleValue())) } /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], preservesPartitioning: Boolean): - JavaPairRDD[K2, V2] = { + def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], + preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) JavaPairRDD.fromRDD(rdd.mapPartitions(fn, preservesPartitioning))(f.keyType(), f.valueType()) } @@ -294,7 +297,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Reduces the elements of this RDD using the specified commutative and associative binary operator. + * Reduces the elements of this RDD using the specified commutative and associative binary + * operator. */ def reduce(f: JFunction2[T, T, T]): T = rdd.reduce(f) 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 5a426b983519c86a956f4e5c2a50593d66dc68f1..22dc9c9e2ecfef936254d8c8d611463f70996263 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 @@ -362,15 +362,15 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork doubleAccumulator(initialValue) /** - * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" + * values to using the `add` method. Only the master can access the accumulator's `value`. */ def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = sc.accumulator(initialValue)(accumulatorParam) /** - * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks can - * "add" values with `add`. Only the master can access the accumuable's `value`. + * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks + * can "add" values with `add`. Only the master can access the accumuable's `value`. */ def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = sc.accumulable(initialValue)(param) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala index 2be4e323bec9808338e8d48fd3463551ef504f69..35eca62ecd5869e4869b4770d75fe4d8277347ce 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala @@ -23,7 +23,8 @@ import org.apache.spark.Partitioner import org.apache.spark.util.Utils /** - * A [[org.apache.spark.Partitioner]] that performs handling of long-valued keys, for use by the Python API. + * A [[org.apache.spark.Partitioner]] that performs handling of long-valued keys, for use by the + * Python API. * * Stores the unique id() of the Python-side partitioning function so that it is incorporated into * equality comparisons. Correctness requires that the id is a unique identifier for the 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 9cbd26b607b2494d0cce6d9228e3fcb498dcf9a0..33667a998ed417003f4ddf808bb85c2de57ebbb1 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 @@ -91,8 +91,9 @@ private[spark] class PythonRDD[T: ClassTag]( // Kill the Python worker process: worker.shutdownOutput() case e: IOException => - // This can happen for legitimate reasons if the Python code stops returning data before we are done - // passing elements through, e.g., for take(). Just log a message to say it happened. + // This can happen for legitimate reasons if the Python code stops returning data + // before we are done passing elements through, e.g., for take(). Just log a message + // to say it happened. logInfo("stdin writer to Python finished early") logDebug("stdin writer to Python finished early", e) } @@ -132,7 +133,8 @@ private[spark] class PythonRDD[T: ClassTag]( val init = initTime - bootTime val finish = finishTime - initTime val total = finishTime - startTime - logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) + logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, + init, finish)) read case SpecialLengths.PYTHON_EXCEPTION_THROWN => // Signals that an exception has been thrown in python @@ -184,7 +186,7 @@ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends override def compute(split: Partition, context: TaskContext) = prev.iterator(split, context).grouped(2).map { case Seq(a, b) => (Utils.deserializeLongValue(a), b) - case x => throw new SparkException("PairwiseRDD: unexpected value: " + x) + case x => throw new SparkException("PairwiseRDD: unexpected value: " + x) } val asJavaPairRDD : JavaPairRDD[Long, Array[Byte]] = JavaPairRDD.fromRDD(this) } @@ -274,7 +276,8 @@ private[spark] object PythonRDD { } -private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { +private +class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8") } 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 d351dfc1f56a2d447085f01f061d9a11fbf03e71..ec997255d59a1aa91fbdad0e84c89f65b3749f52 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -187,8 +187,9 @@ extends Logging { val bais = new ByteArrayInputStream(byteArray) var blockNum = (byteArray.length / BLOCK_SIZE) - if (byteArray.length % BLOCK_SIZE != 0) + if (byteArray.length % BLOCK_SIZE != 0) { blockNum += 1 + } var retVal = new Array[TorrentBlock](blockNum) var blockID = 0 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 db67c6d1bb55c58069a23a7d2d3abb01a4d2a1ea..3db970ca73b923d1f60b574085938cf07f0f65fe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -101,16 +101,16 @@ private[spark] class ClientArguments(args: Array[String]) { // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars // separately similar to in the YARN client. val usage = - s""" - |Usage: DriverClient [options] launch <active-master> <jar-url> <main-class> [driver options] - |Usage: DriverClient kill <active-master> <driver-id> - | - |Options: - | -c CORES, --cores CORES Number of cores to request (default: $defaultCores) - | -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $defaultMemory) - | -s, --supervise Whether to restart the driver on failure - | -v, --verbose Print more debugging output - """.stripMargin + s""" + |Usage: DriverClient [options] launch <active-master> <jar-url> <main-class> [driver options] + |Usage: DriverClient kill <active-master> <driver-id> + | + |Options: + | -c CORES, --cores CORES Number of cores to request (default: $defaultCores) + | -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $defaultMemory) + | -s, --supervise Whether to restart the driver on failure + | -v, --verbose Print more debugging output + """.stripMargin System.err.println(usage) System.exit(exitCode) } 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 4dfb19ed8adb6865c6a5912eada5f292d5b3d624..7de7c4864ee549d955d794b80add9fcab3ef7c93 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -1,20 +1,18 @@ /* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * http://www.apache.org/licenses/LICENSE-2.0 * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 @@ -306,7 +304,8 @@ private[spark] object FaultToleranceTest extends App with Logging { } } - logInfo("Ran %s tests, %s passed and %s failed".format(numPassed+numFailed, numPassed, numFailed)) + logInfo("Ran %s tests, %s passed and %s failed".format(numPassed + numFailed, numPassed, + numFailed)) } private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index ffc0cb09032fb6e324fa33e828f256a7672b1033..488843a32c167b7dd8903c263b3c4c2cbdfef215 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -33,7 +33,8 @@ import scala.collection.mutable.ArrayBuffer * fault recovery without spinning up a lot of processes. */ private[spark] -class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) extends Logging { +class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) + extends Logging { private val localHostname = Utils.localHostName() private val masterActorSystems = ArrayBuffer[ActorSystem]() diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 1415e2f3d1886974cdc6a725b973ffbbe6ccc7e8..8901806de92621e1f53a3e3f3b18cb535c081183 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -132,7 +132,8 @@ private[spark] class AppClient( case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) => val fullId = appId + "/" + id - logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) + logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, + cores)) listener.executorAdded(fullId, workerId, hostPort, cores, memory) case ExecutorUpdated(id, state, message, exitStatus) => diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala index 55d4ef1b31aaacbe49878ba4bb3cabebfc7946a2..2f2cbd182c967526b6746cc8031de9f9bdc56917 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala @@ -33,7 +33,8 @@ private[spark] trait AppClientListener { /** Dead means that we couldn't find any Masters to connect to, and have given up. */ def dead(): Unit - def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit + def executorAdded( + fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit } 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 2ef167ffc00f0000f2fc51b4cfdbce99440e655f..82bf655212fcc2a30afef218d1d72b955abb80f4 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 @@ -149,10 +149,11 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act override def receive = { case ElectedLeader => { val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData() - state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) + state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) { RecoveryState.ALIVE - else + } else { RecoveryState.RECOVERING + } logInfo("I have been elected leader! New state: " + state) if (state == RecoveryState.RECOVERING) { beginRecovery(storedApps, storedDrivers, storedWorkers) @@ -165,7 +166,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act System.exit(0) } - case RegisterWorker(id, workerHost, workerPort, cores, memory, workerWebUiPort, publicAddress) => { + case RegisterWorker(id, workerHost, workerPort, cores, memory, workerWebUiPort, publicAddress) + => { logInfo("Registering worker %s:%d with %d cores, %s RAM".format( host, workerPort, cores, Utils.megabytesToString(memory))) if (state == RecoveryState.STANDBY) { @@ -181,9 +183,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act schedule() } else { val workerAddress = worker.actor.path.address - logWarning("Worker registration failed. Attempted to re-register worker at same address: " + - workerAddress) - sender ! RegisterWorkerFailed("Attempted to re-register worker at same address: " + workerAddress) + logWarning("Worker registration failed. Attempted to re-register worker at same " + + "address: " + workerAddress) + sender ! RegisterWorkerFailed("Attempted to re-register worker at same address: " + + workerAddress) } } } @@ -641,8 +644,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act worker.id, WORKER_TIMEOUT/1000)) removeWorker(worker) } else { - if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) + if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT)) { workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it + } } } } 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/IndexPage.scala index a9af8df5525d68375b552be8526e3f1a685da111..64ecf22399e395f620863b51d722553008b67444 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/IndexPage.scala @@ -57,7 +57,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { val completedApps = state.completedApps.sortBy(_.endTime).reverse val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps) - val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory", "Main Class") + val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory", + "Main Class") val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers) val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse @@ -103,13 +104,14 @@ private[spark] class IndexPage(parent: MasterWebUI) { </div> <div> - {if (hasDrivers) - <div class="row-fluid"> - <div class="span12"> - <h4> Running Drivers </h4> - {activeDriversTable} - </div> - </div> + {if (hasDrivers) { + <div class="row-fluid"> + <div class="span12"> + <h4> Running Drivers </h4> + {activeDriversTable} + </div> + </div> + } } </div> @@ -121,13 +123,14 @@ private[spark] class IndexPage(parent: MasterWebUI) { </div> <div> - {if (hasDrivers) - <div class="row-fluid"> - <div class="span12"> - <h4> Completed Drivers </h4> - {completedDriversTable} - </div> - </div> + {if (hasDrivers) { + <div class="row-fluid"> + <div class="span12"> + <h4> Completed Drivers </h4> + {completedDriversTable} + </div> + </div> + } } </div>; @@ -175,7 +178,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { <tr> <td>{driver.id} </td> <td>{driver.submitDate}</td> - <td>{driver.worker.map(w => <a href={w.webUiAddress}>{w.id.toString}</a>).getOrElse("None")}</td> + <td>{driver.worker.map(w => <a href={w.webUiAddress}>{w.id.toString}</a>).getOrElse("None")} + </td> <td>{driver.state}</td> <td sorttable_customkey={driver.desc.cores.toString}> {driver.desc.cores} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 460883ec7ae24bbf1c16025f79f73d80eb41a833..f411eb9cec89f288802f4109866024e4110a1f65 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -49,7 +49,8 @@ object CommandUtils extends Logging { val libraryOpts = getEnv("SPARK_LIBRARY_PATH", command) .map(p => List("-Djava.library.path=" + p)) .getOrElse(Nil) - val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil) + val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")) + .map(Utils.splitCommandString).getOrElse(Nil) val userOpts = getEnv("SPARK_JAVA_OPTS", command).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 6f6c101547c3c83a08956c75d8035657531e9235..a26e47950a0ece2e4c9a2fe246827915a1825c18 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -45,4 +45,4 @@ object DriverWrapper { System.exit(-1) } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 1dc39c450ea1634a3fdd2feedeb9f799bbd1c7dd..530c147000904094bd579bc8cf12d8ce207fd46b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -69,4 +69,4 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor case e => logWarning(s"Received unexpected actor system event: $e") } -} \ No newline at end of file +} 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 925c6fb1832d7eff6adea8d51c91baf937ef0a97..3089acffb8d98e3d92c5936deac2b791e04c0db3 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 @@ -84,7 +84,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { {runningExecutorTable} </div> </div> - + // scalastyle:off <div> {if (hasDrivers) <div class="row-fluid"> <!-- Running Drivers --> @@ -113,7 +113,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { </div> } </div>; - + // scalastyle:on UIUtils.basicSparkPage(content, "Spark Worker at %s:%s".format( workerState.host, workerState.port)) } @@ -133,10 +133,10 @@ private[spark] class IndexPage(parent: WorkerWebUI) { </ul> </td> <td> - <a href={"logPage?appId=%s&executorId=%s&logType=stdout" - .format(executor.appId, executor.execId)}>stdout</a> - <a href={"logPage?appId=%s&executorId=%s&logType=stderr" - .format(executor.appId, executor.execId)}>stderr</a> + <a href={"logPage?appId=%s&executorId=%s&logType=stdout" + .format(executor.appId, executor.execId)}>stdout</a> + <a href={"logPage?appId=%s&executorId=%s&logType=stderr" + .format(executor.appId, executor.execId)}>stderr</a> </td> </tr> 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 c23b75d75745672aa9f08324aeabfbf5d6fb5f23..86688e44242a9b04a33aad4a0a6b00cbbe8de803 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 @@ -187,7 +187,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val logPageLength = math.min(byteLength, maxBytes) - val endByte = math.min(startByte+logPageLength, logLength) + val endByte = math.min(startByte + logPageLength, logLength) (startByte, endByte) } 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 f7efd74e1b04393d39d7287c4d5e127192628858..989d666f156005000947fa02625e7e82c0c3de1c 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -205,7 +205,7 @@ private[spark] class Executor( } attemptedTask = Some(task) - logDebug("Task " + taskId +"'s epoch is " + task.epoch) + logDebug("Task " + taskId + "'s epoch is " + task.epoch) env.mapOutputTracker.updateEpoch(task.epoch) // Run the actual task and measure its runtime. @@ -233,7 +233,8 @@ private[spark] class Executor( val accumUpdates = Accumulators.values - val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.getOrElse(null)) + val directResult = new DirectTaskResult(valueBytes, accumUpdates, + task.metrics.getOrElse(null)) val serializedDirectResult = ser.serialize(directResult) logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit) val serializedResult = { 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 e5c9bbbe2874e66b8fff8a16126bd1623138d8f2..210f3dbeebaca6a56bd71d42d4652985a6f4b1e9 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -50,10 +50,11 @@ object ExecutorExitCode { "Failed to create local directory (bad spark.local.dir?)" case _ => "Unknown executor exit code (" + exitCode + ")" + ( - if (exitCode > 128) + if (exitCode > 128) { " (died from signal " + (exitCode - 128) + "?)" - else + } else { "" + } ) } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 97176e4f5b727c78fbc1a72cdf949df1c8fe8290..c2e973e1738d428356e1402c6f4f64df568eb4fd 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -55,7 +55,8 @@ class ExecutorSource(val executor: Executor, executorId: String) extends Source override def getValue: Int = executor.threadPool.getPoolSize() }) - // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + // Gauge got executor thread pool's largest number of threads that have ever simultaneously + // been in th pool metricRegistry.register(MetricRegistry.name("threadpool", "maxPool_size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 0c8f4662a5f3a945bb1ea93b4870e42d8c9dcf3e..455339943f42d252091b28e6a18d3e04c619a43e 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -64,7 +64,8 @@ class TaskMetrics extends Serializable { var shuffleReadMetrics: Option[ShuffleReadMetrics] = None /** - * If this task writes to shuffle output, metrics on the written shuffle data will be collected here + * If this task writes to shuffle output, metrics on the written shuffle data will be collected + * here */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 9930537b34db01ac0e619cdf482bbf4bd9ab7fdb..de233e416a9dcd6df264cd8a4aadf5a1049a0f24 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -56,7 +56,8 @@ import org.apache.spark.metrics.source.Source * wild card "*" can be used to replace instance name, which means all the instances will have * this property. * - * [sink|source] means this property belongs to source or sink. This field can only be source or sink. + * [sink|source] means this property belongs to source or sink. This field can only be + * source or sink. * * [name] specify the name of sink or source, it is custom defined. * 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 cba8477ed572336bdcd74d3119adba1948794570..ae2007e41b77fd4c289a0080586c9128c362acf8 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -211,7 +211,6 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } return chunk } else { - /*logInfo("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "]")*/ message.finishTime = System.currentTimeMillis logDebug("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "] in " + message.timeTaken ) @@ -238,7 +237,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, message.startTime = System.currentTimeMillis } logTrace( - "Sending chunk from [" + message+ "] to [" + getRemoteConnectionManagerId() + "]") + "Sending chunk from [" + message + "] to [" + getRemoteConnectionManagerId() + "]") return chunk } else { message.finishTime = System.currentTimeMillis @@ -349,8 +348,8 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, outbox.getChunk() match { case Some(chunk) => { val buffers = chunk.buffers - // If we have 'seen' pending messages, then reset flag - since we handle that as normal - // registering of event (below) + // If we have 'seen' pending messages, then reset flag - since we handle that as + // normal registering of event (below) if (needForceReregister && buffers.exists(_.remaining() > 0)) resetForceReregister() currentBuffers ++= buffers } @@ -404,7 +403,8 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } } catch { case e: Exception => - logError("Exception while reading SendingConnection to " + getRemoteConnectionManagerId(), e) + logError("Exception while reading SendingConnection to " + getRemoteConnectionManagerId(), + e) callOnExceptionCallback(e) close() } 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 e6e01783c889524c561f0addc6f136c91f6eae57..24d0a7deb57d04e9b123d4830ac24917589ee008 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -65,7 +65,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable]()) - // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap + // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : + // which should be executed asap private val handleConnectExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.connect.threads.min", 1), conf.getInt("spark.core.connection.connect.threads.max", 8), @@ -73,8 +74,10 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi new LinkedBlockingDeque[Runnable]()) private val serverChannel = ServerSocketChannel.open() - private val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] - private val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] + private val connectionsByKey = new HashMap[SelectionKey, Connection] + with SynchronizedMap[SelectionKey, Connection] + private val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] + with SynchronizedMap[ConnectionManagerId, SendingConnection] private val messageStatuses = new HashMap[Int, MessageStatus] private val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)] private val registerRequests = new SynchronizedQueue[SendingConnection] @@ -173,7 +176,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi if (conn == null) return // prevent other events from being triggered - // Since we are still trying to connect, we do not need to do the additional steps in triggerWrite + // Since we are still trying to connect, we do not need to do the additional steps in + // triggerWrite conn.changeConnectionKeyInterest(0) handleConnectExecutor.execute(new Runnable { @@ -188,8 +192,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi } // fallback to previous behavior : we should not really come here since this method was - // triggered since channel became connectable : but at times, the first finishConnect need not - // succeed : hence the loop to retry a few 'times'. + // triggered since channel became connectable : but at times, the first finishConnect need + // not succeed : hence the loop to retry a few 'times'. conn.finishConnect(true) } } ) @@ -258,8 +262,9 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " " } - logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId() + - "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") + logTrace("Changed key for connection to [" + + connection.getRemoteConnectionManagerId() + "] changed from [" + + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") } } } else { @@ -282,7 +287,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi try { selector.select() } catch { - // Explicitly only dealing with CancelledKeyException here since other exceptions should be dealt with differently. + // Explicitly only dealing with CancelledKeyException here since other exceptions + // should be dealt with differently. case e: CancelledKeyException => { // Some keys within the selectors list are invalid/closed. clear them. val allKeys = selector.keys().iterator() @@ -310,7 +316,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi } if (selectedKeysCount == 0) { - logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys") + logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + + " keys") } if (selectorThread.isInterrupted) { logInfo("Selector thread was interrupted!") @@ -341,7 +348,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi throw new CancelledKeyException() } } catch { - // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. + // weird, but we saw this happening - even though key.isValid was true, + // key.isAcceptable would throw CancelledKeyException. case e: CancelledKeyException => { logInfo("key already cancelled ? " + key, e) triggerForceCloseByException(key, e) @@ -437,9 +445,10 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi assert (sendingConnectionManagerId == remoteConnectionManagerId) messageStatuses.synchronized { - for (s <- messageStatuses.values if s.connectionManagerId == sendingConnectionManagerId) { - logInfo("Notifying " + s) - s.synchronized { + for (s <- messageStatuses.values if + s.connectionManagerId == sendingConnectionManagerId) { + logInfo("Notifying " + s) + s.synchronized { s.attempted = true s.acked = false s.markDone() @@ -458,7 +467,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi } def handleConnectionError(connection: Connection, e: Exception) { - logInfo("Handling connection error on connection to " + connection.getRemoteConnectionManagerId()) + logInfo("Handling connection error on connection to " + + connection.getRemoteConnectionManagerId()) removeConnection(connection) } @@ -495,7 +505,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi status } case None => { - throw new Exception("Could not find reference for received ack message " + message.id) + throw new Exception("Could not find reference for received ack message " + + message.id) null } } @@ -517,7 +528,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi if (ackMessage.isDefined) { if (!ackMessage.get.isInstanceOf[BufferMessage]) { - logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + ackMessage.get.getClass()) + logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + + ackMessage.get.getClass()) } else if (!ackMessage.get.asInstanceOf[BufferMessage].hasAckId) { logDebug("Response to " + bufferMessage + " does not have ack id set") ackMessage.get.asInstanceOf[BufferMessage].ackId = bufferMessage.id @@ -535,14 +547,16 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private def sendMessage(connectionManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { - val inetSocketAddress = new InetSocketAddress(connectionManagerId.host, connectionManagerId.port) + val inetSocketAddress = new InetSocketAddress(connectionManagerId.host, + connectionManagerId.port) val newConnection = new SendingConnection(inetSocketAddress, selector, connectionManagerId) registerRequests.enqueue(newConnection) newConnection } - // I removed the lookupKey stuff as part of merge ... should I re-add it ? We did not find it useful in our test-env ... - // If we do re-add it, we should consistently use it everywhere I guess ? + // I removed the lookupKey stuff as part of merge ... should I re-add it ? We did not find it + // useful in our test-env ... If we do re-add it, we should consistently use it everywhere I + // guess ? val connection = connectionsById.getOrElseUpdate(connectionManagerId, startNewConnection()) message.senderAddress = id.toSocketAddress() logDebug("Sending [" + message + "] to [" + connectionManagerId + "]") @@ -558,15 +572,17 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi def sendMessageReliably(connectionManagerId: ConnectionManagerId, message: Message) : Future[Option[Message]] = { val promise = Promise[Option[Message]] - val status = new MessageStatus(message, connectionManagerId, s => promise.success(s.ackMessage)) - messageStatuses.synchronized { + val status = new MessageStatus( + message, connectionManagerId, s => promise.success(s.ackMessage)) + messageStatuses.synchronized { messageStatuses += ((message.id, status)) } sendMessage(connectionManagerId, message) promise.future } - def sendMessageReliablySync(connectionManagerId: ConnectionManagerId, message: Message): Option[Message] = { + def sendMessageReliablySync(connectionManagerId: ConnectionManagerId, + message: Message): Option[Message] = { Await.result(sendMessageReliably(connectionManagerId, message), Duration.Inf) } @@ -656,7 +672,8 @@ private[spark] object ConnectionManager { val tput = mb * 1000.0 / ms println("--------------------------") println("Started at " + startTime + ", finished at " + finishTime) - println("Sent " + count + " messages of size " + size + " in " + ms + " ms (" + tput + " MB/s)") + println("Sent " + count + " messages of size " + size + " in " + ms + " ms " + + "(" + tput + " MB/s)") println("--------------------------") println() } @@ -667,7 +684,8 @@ private[spark] object ConnectionManager { println("--------------------------") val size = 10 * 1024 * 1024 val count = 10 - val buffers = Array.tabulate(count)(i => ByteBuffer.allocate(size * (i + 1)).put(Array.tabulate[Byte](size * (i + 1))(x => x.toByte))) + val buffers = Array.tabulate(count)(i => ByteBuffer.allocate(size * (i + 1)).put( + Array.tabulate[Byte](size * (i + 1))(x => x.toByte))) buffers.foreach(_.flip) val mb = buffers.map(_.remaining).reduceLeft(_ + _) / 1024.0 / 1024.0 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 4f5742d29b3677a7b9e5d3b22528cda0d9cbafd0..820045aa21813b3f234bb9812bc36206d38cab6c 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -30,14 +30,14 @@ import scala.concurrent.duration._ private[spark] object ConnectionManagerTest extends Logging{ def main(args: Array[String]) { - //<mesos cluster> - the master URL - //<slaves file> - a list slaves to run connectionTest on - //[num of tasks] - the number of parallel tasks to be initiated default is number of slave hosts - //[size of msg in MB (integer)] - the size of messages to be sent in each task, default is 10 - //[count] - how many times to run, default is 3 - //[await time in seconds] : await time (in seconds), default is 600 + // <mesos cluster> - the master URL <slaves file> - a list slaves to run connectionTest on + // [num of tasks] - the number of parallel tasks to be initiated default is number of slave + // hosts [size of msg in MB (integer)] - the size of messages to be sent in each task, + // default is 10 [count] - how many times to run, default is 3 [await time in seconds] : + // await time (in seconds), default is 600 if (args.length < 2) { - println("Usage: ConnectionManagerTest <mesos cluster> <slaves file> [num of tasks] [size of msg in MB (integer)] [count] [await time in seconds)] ") + println("Usage: ConnectionManagerTest <mesos cluster> <slaves file> [num of tasks] " + + "[size of msg in MB (integer)] [count] [await time in seconds)] ") System.exit(1) } @@ -56,7 +56,8 @@ private[spark] object ConnectionManagerTest extends Logging{ 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 + ", msg size = " + size/1024/1024 + " MB, awaitTime = " + awaitTime) + println("Running " + count + " rounds of test: " + "parallel tasks = " + tasknum + ", " + + "msg size = " + size/1024/1024 + " MB, awaitTime = " + awaitTime) val slaveConnManagerIds = sc.parallelize(0 until tasknum, tasknum).map( i => SparkEnv.get.connectionManager.id).collect() println("\nSlave ConnectionManagerIds") @@ -76,7 +77,8 @@ private[spark] object ConnectionManagerTest extends Logging{ buffer.flip val startTime = System.currentTimeMillis - val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map(slaveConnManagerId => { + val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map(slaveConnManagerId => + { val bufferMessage = Message.createBufferMessage(buffer.duplicate) logInfo("Sending [" + bufferMessage + "] to [" + slaveConnManagerId + "]") connManager.sendMessageReliably(slaveConnManagerId, bufferMessage) @@ -87,7 +89,8 @@ private[spark] object ConnectionManagerTest extends Logging{ 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 * 1000.0) + " MB/s" + val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * + 1000.0) + " MB/s" logInfo(resultStr) resultStr }).collect() diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index dcbd183c88d09b4aad12b85674207adf8fce60ac..9e03956ba0df979c3a61c06539765627a6bc35dd 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -52,17 +52,20 @@ private[spark] object SenderTest { val dataMessage = Message.createBufferMessage(buffer.duplicate) val startTime = System.currentTimeMillis /*println("Started timer at " + startTime)*/ - val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) match { - case Some(response) => - val buffer = response.asInstanceOf[BufferMessage].buffers(0) - new String(buffer.array) - case None => "none" - } + val responseStr = + manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) match { + case Some(response) => + val buffer = response.asInstanceOf[BufferMessage].buffers(0) + new String(buffer.array) + case None => "none" + } val finishTime = System.currentTimeMillis val mb = size / 1024.0 / 1024.0 val ms = finishTime - startTime - /*val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms at " + (mb / ms * 1000.0) + " MB/s"*/ - val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms (" + (mb / ms * 1000.0).toInt + "MB/s) | Response = " + responseStr + // val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms at " + (mb / ms + // * 1000.0) + " MB/s" + val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms (" + (mb / ms * + 1000.0).toInt + "MB/s) | Response = " + responseStr println(resultStr) }) } diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 30e578dd93e8d642b3c8404a915d07fe95a68792..8f9d1d5a84c36c6f28a955de950d711ad8a6ddf5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -60,7 +60,8 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String) checkpointData.get.cpFile = Some(checkpointPath) override def getPreferredLocations(split: Partition): Seq[String] = { - val status = fs.getFileStatus(new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index))) + val status = fs.getFileStatus(new Path(checkpointPath, + CheckpointRDD.splitIdToFile(split.index))) val locations = fs.getFileBlockLocations(status, 0, status.getLen) locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost") } 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 cefcc3d2d9420178c816937a6e8c615143debd8a..42e1ef8375284bd82333de055aafd1922948e304 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -197,8 +197,9 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc // return the next preferredLocation of some partition of the RDD def next(): (String, Partition) = { - if (it.hasNext) + if (it.hasNext) { it.next() + } else { it = resetIterator() // ran out of preferred locations, reset and rotate to the beginning it.next() @@ -290,8 +291,10 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc val r1 = rnd.nextInt(groupArr.size) val r2 = rnd.nextInt(groupArr.size) val minPowerOfTwo = if (groupArr(r1).size < groupArr(r2).size) groupArr(r1) else groupArr(r2) - if (prefPart== None) // if no preferred locations, just use basic power of two - return minPowerOfTwo + if (prefPart == None) { + // if no preferred locations, just use basic power of two + return minPowerOfTwo + } val prefPartActual = prefPart.get 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 688c310ee9caf37f117c728b3b9a27adb778ef7f..20713b4249b5e5751459d5085b5f434d1fe53281 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -37,8 +37,8 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { } /** - * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and count - * of the RDD's elements in one operation. + * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and + * count of the RDD's elements in one operation. */ def stats(): StatCounter = { self.mapPartitions(nums => Iterator(StatCounter(nums))).reduce((a, b) => a.merge(b)) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 370061492da91666634113e9d6cef0c3517340bc..10d519e6971f1321602de95bdbc7038fb81fe18f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -705,7 +705,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + - valueClass.getSimpleName+ ")") + valueClass.getSimpleName + ")") val writer = new SparkHadoopWriter(conf) writer.preSetup() diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 09d0a8189d25c2c9cc8cb08737e3e9a3740ac1c6..56c7777600a6a8a2ec2c5e77c7f3b8071e5f6344 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -39,7 +39,8 @@ private[spark] class ParallelCollectionPartition[T: ClassTag]( override def hashCode(): Int = (41 * (41 + rddId) + slice).toInt override def equals(other: Any): Boolean = other match { - case that: ParallelCollectionPartition[_] => (this.rddId == that.rddId && this.slice == that.slice) + case that: ParallelCollectionPartition[_] => (this.rddId == that.rddId && + this.slice == that.slice) case _ => false } 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 4c625d062eb9bef5abaf71a180af13cafaf7f18c..f4364329a3a71c11843c7c574c1ead063ed841d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -23,8 +23,8 @@ import org.apache.spark.{TaskContext, OneToOneDependency, SparkContext, Partitio /** - * Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of corresponding partitions - * of parent RDDs. + * Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of + * corresponding partitions of parent RDDs. */ private[spark] class PartitionerAwareUnionRDDPartition( diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index bc688110f47362cef8cdb5bf9e92d3e71b22f9c9..73e8769c0981d75b416f51cde6446bee91869d88 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -35,10 +35,10 @@ private[spark] object CheckpointState extends Enumeration { } /** - * This class contains all the information related to RDD checkpointing. Each instance of this class - * is associated with a RDD. It manages process of checkpointing of the associated RDD, as well as, - * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations - * of the checkpointed RDD. + * This class contains all the information related to RDD checkpointing. Each instance of this + * class is associated with a RDD. It manages process of checkpointing of the associated RDD, + * as well as, manages the post-checkpoint state by providing the updated partitions, + * iterator and preferred locations of the checkpointed RDD. */ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) extends Logging with Serializable { @@ -97,7 +97,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) if (newRDD.partitions.size != rdd.partitions.size) { throw new SparkException( - "Checkpoint RDD " + newRDD + "("+ newRDD.partitions.size + ") has different " + + "Checkpoint RDD " + newRDD + "(" + newRDD.partitions.size + ") has different " + "number of partitions than original RDD " + rdd + "(" + rdd.partitions.size + ")") } diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 2d1bd5b4813c9dc77973c6fe9f6cf59b613f90b6..c9b4c768a98b405cdab7298b6c400fee8c597599 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -71,7 +71,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass) val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass) - logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) + logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + + valueClass.getSimpleName + ")" ) val format = classOf[SequenceFileOutputFormat[Writable, Writable]] val jobConf = new JobConf(self.context.hadoopConfiguration) if (!convertKey && !convertValue) { 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 21d16fabefaa55b63e739cbb69b2cc0b28070c93..80211541a6a633ec5b6593dd6d6c09dcde12e697 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1082,8 +1082,9 @@ class DAGScheduler( case n: NarrowDependency[_] => for (inPart <- n.getParents(partition)) { val locs = getPreferredLocs(n.rdd, inPart) - if (locs != Nil) + if (locs != Nil) { return locs + } } case _ => } diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index cc10cc0849bc78ff3440d9e7e0323efee398620c..23447f1bbf852956858768950cd5c395cccb4bb6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -33,7 +33,7 @@ import scala.collection.JavaConversions._ * Parses and holds information about inputFormat (and files) specified as a parameter. */ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], - val path: String) extends Logging { + val path: String) extends Logging { var mapreduceInputFormat: Boolean = false var mapredInputFormat: Boolean = false @@ -41,7 +41,8 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl validate() override def toString: String = { - "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", path : " + path + "InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", " + + "path : " + path } override def hashCode(): Int = { @@ -50,8 +51,8 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl hashCode } - // Since we are not doing canonicalization of path, this can be wrong : like relative vs absolute path - // .. which is fine, this is best case effort to remove duplicates - right ? + // Since we are not doing canonicalization of path, this can be wrong : like relative vs + // absolute path .. which is fine, this is best case effort to remove duplicates - right ? override def equals(other: Any): Boolean = other match { case that: InputFormatInfo => { // not checking config - that should be fine, right ? @@ -65,22 +66,26 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl logDebug("validate InputFormatInfo : " + inputFormatClazz + ", path " + path) try { - if (classOf[org.apache.hadoop.mapreduce.InputFormat[_, _]].isAssignableFrom(inputFormatClazz)) { + if (classOf[org.apache.hadoop.mapreduce.InputFormat[_, _]].isAssignableFrom( + inputFormatClazz)) { logDebug("inputformat is from mapreduce package") mapreduceInputFormat = true } - else if (classOf[org.apache.hadoop.mapred.InputFormat[_, _]].isAssignableFrom(inputFormatClazz)) { + else if (classOf[org.apache.hadoop.mapred.InputFormat[_, _]].isAssignableFrom( + inputFormatClazz)) { logDebug("inputformat is from mapred package") mapredInputFormat = true } else { throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + - " is NOT a supported input format ? does not implement either of the supported hadoop api's") + " is NOT a supported input format ? does not implement either of the supported hadoop " + + "api's") } } catch { case e: ClassNotFoundException => { - throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + " cannot be found ?", e) + throw new IllegalArgumentException("Specified inputformat " + inputFormatClazz + + " cannot be found ?", e) } } } @@ -125,8 +130,8 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl } private def findPreferredLocations(): Set[SplitInfo] = { - logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat + - ", inputFormatClazz : " + inputFormatClazz) + logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + + mapredInputFormat + ", inputFormatClazz : " + inputFormatClazz) if (mapreduceInputFormat) { prefLocsFromMapreduceInputFormat() } @@ -150,8 +155,8 @@ object InputFormatInfo { c) Compute rack info for each host and update rack -> count map based on (b). d) Allocate nodes based on (c) e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node - (even if data locality on that is very high) : this is to prevent fragility of job if a single - (or small set of) hosts go down. + (even if data locality on that is very high) : this is to prevent fragility of job if a + single (or small set of) hosts go down. go to (a) until required nodes are allocated. @@ -159,7 +164,8 @@ object InputFormatInfo { PS: I know the wording here is weird, hopefully it makes some sense ! */ - def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] = { + def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] + = { val nodeToSplit = new HashMap[String, HashSet[SplitInfo]] for (inputSplit <- formats) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index f8fa5a9f7a5903384c4514883701d37c82809827..b909b66a5de768d6bbe430bbefd17518d6078b67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -45,10 +45,11 @@ class JobLogger(val user: String, val logDirName: String) String.valueOf(System.currentTimeMillis())) private val logDir = - if (System.getenv("SPARK_LOG_DIR") != null) + if (System.getenv("SPARK_LOG_DIR") != null) { System.getenv("SPARK_LOG_DIR") - else + } else { "/tmp/spark-%s".format(user) + } private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] private val stageIDToJobID = new HashMap[Int, Int] @@ -116,7 +117,7 @@ class JobLogger(val user: String, val logDirName: String) var writeInfo = info if (withTime) { val date = new Date(System.currentTimeMillis()) - writeInfo = DATE_FORMAT.format(date) + ": " +info + writeInfo = DATE_FORMAT.format(date) + ": " + info } jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo)) } @@ -235,7 +236,8 @@ class JobLogger(val user: String, val logDirName: String) * @param stage Root stage of the job * @param indent Indent number before info, default is 0 */ - protected def recordStageDepGraph(jobID: Int, stage: Stage, idSet: HashSet[Int], indent: Int = 0) { + protected def recordStageDepGraph(jobID: Int, stage: Stage, idSet: HashSet[Int], indent: Int = 0) + { val stageInfo = if (stage.isShuffleMap) { "STAGE_ID=" + stage.id + " MAP_STAGE SHUFFLE_ID=" + stage.shuffleDep.get.shuffleId } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index c381348a8d424af31171f1d8d07ab7fb931afc8d..d94f6ad924260c40a07cd27a77fb3e2681f2046b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -23,4 +23,5 @@ package org.apache.spark.scheduler private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult -private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage]) extends JobResult +private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage]) + extends JobResult 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 28f3ba53b84253e3b5029c686d3ef9c7936eadf9..0544f81f1ce86a5840d6b38cab2974ce8954d656 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -36,7 +36,8 @@ private[spark] object ResultTask { val metadataCleaner = new MetadataCleaner( MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf) - def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = { + def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _) + : Array[Byte] = { synchronized { val old = serializedInfoCache.get(stageId).orNull if (old != null) { @@ -55,7 +56,8 @@ private[spark] object ResultTask { } } - def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = { + 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() 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 3cf995ea74244c2440d101b5a247fa6896454bc3..a546193d5b49ae6f56b28311c0b1c138aa3b8385 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -148,6 +148,6 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) } } parentPool.addSchedulable(manager) - logInfo("Added task set " + manager.name + " tasks to pool "+poolName) + logInfo("Added task set " + manager.name + " tasks to pool " + poolName) } } 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 d8e97c3b7c7b0284fa8afa225b1375236322cf9e..d25f0a63547e629b7afbc10855b458b34285d698 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -37,8 +37,8 @@ case class SparkListenerTaskGettingResult( case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvents -case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], properties: Properties = null) - extends SparkListenerEvents +case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], + properties: Properties = null) extends SparkListenerEvents case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) extends SparkListenerEvents @@ -99,11 +99,14 @@ class StatsReportListener extends SparkListener with Logging { showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) //shuffle write - showBytesDistribution("shuffle bytes written:",(_,metric) => metric.shuffleWriteMetrics.map{_.shuffleBytesWritten}) + showBytesDistribution("shuffle bytes written:", + (_,metric) => metric.shuffleWriteMetrics.map{_.shuffleBytesWritten}) //fetch & io - showMillisDistribution("fetch wait time:",(_, metric) => metric.shuffleReadMetrics.map{_.fetchWaitTime}) - showBytesDistribution("remote bytes read:", (_, metric) => metric.shuffleReadMetrics.map{_.remoteBytesRead}) + showMillisDistribution("fetch wait time:", + (_, metric) => metric.shuffleReadMetrics.map{_.fetchWaitTime}) + showBytesDistribution("remote bytes read:", + (_, metric) => metric.shuffleReadMetrics.map{_.remoteBytesRead}) showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) //runtime breakdown @@ -111,8 +114,10 @@ class StatsReportListener extends SparkListener with Logging { val runtimePcts = stageCompleted.stage.taskInfos.map{ case (info, metrics) => RuntimePercentage(info.duration, metrics) } - showDistribution("executor (non-fetch) time pct: ", Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") - showDistribution("fetch wait time pct: ", Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%") + showDistribution("executor (non-fetch) time pct: ", + Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") + showDistribution("fetch wait time pct: ", + Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%") showDistribution("other time pct: ", Distribution(runtimePcts.map{_.other * 100}), "%2.0f %%") } @@ -147,7 +152,8 @@ private[spark] object StatsReportListener extends Logging { logInfo("\t" + quantiles.mkString("\t")) } - def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) { + def showDistribution(heading: String, + dOpt: Option[Distribution], formatNumber: Double => String) { dOpt.foreach { d => showDistribution(heading, d, formatNumber)} } @@ -156,7 +162,8 @@ private[spark] object StatsReportListener extends Logging { showDistribution(heading, dOpt, f _) } - def showDistribution(heading:String, format: String, getMetric: (TaskInfo,TaskMetrics) => Option[Double]) + def showDistribution(heading:String, format: String, + getMetric: (TaskInfo,TaskMetrics) => Option[Double]) (implicit stage: SparkListenerStageCompleted) { showDistribution(heading, extractDoubleDistribution(stage, getMetric), format) } @@ -175,7 +182,8 @@ private[spark] object StatsReportListener extends Logging { } def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { - showDistribution(heading, dOpt, (d => StatsReportListener.millisToString(d.toLong)): Double => String) + showDistribution(heading, dOpt, + (d => StatsReportListener.millisToString(d.toLong)): Double => String) } def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) @@ -212,7 +220,7 @@ private object RuntimePercentage { val denom = totalTime.toDouble val fetchTime = metrics.shuffleReadMetrics.map{_.fetchWaitTime} val fetch = fetchTime.map{_ / denom} - val exec = (metrics.executorRunTime - fetchTime.getOrElse(0l)) / denom + val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom val other = 1.0 - (exec + fetch.getOrElse(0d)) RuntimePercentage(exec, fetch, other) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 520c0b29e3536c4681d2d3710ec1d5a47e07cb58..a78b0186b9eab7a4d887bc4152335251315ba946 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -63,8 +63,9 @@ private[spark] class Stage( def addOutputLoc(partition: Int, status: MapStatus) { val prevList = outputLocs(partition) outputLocs(partition) = status :: prevList - if (prevList == Nil) + if (prevList == Nil) { numAvailableOutputs += 1 + } } def removeOutputLoc(partition: Int, bmAddress: BlockManagerId) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index c4d1ad5733b4ce8708244046f10d5fac0fb9b031..8f320e5c7a74bd2db1f44c3b3b224ee3ca658f41 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -29,7 +29,8 @@ import org.apache.spark.executor.TaskMetrics */ class StageInfo( stage: Stage, - val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]() + val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = + mutable.Buffer[(TaskInfo, TaskMetrics)]() ) { val stageId = stage.id /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 3c22edd5248f403190a2f543597728d08dba92a2..91c27d7b8e9d7aeda5c0e33d5a1ef0a47cbed91c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -70,16 +70,17 @@ class TaskInfo( def running: Boolean = !finished def status: String = { - if (running) + if (running) { "RUNNING" - else if (gettingResult) + } else if (gettingResult) { "GET RESULT" - else if (failed) + } else if (failed) { "FAILED" - else if (successful) + } else if (successful) { "SUCCESS" - else + } else { "UNKNOWN" + } } def duration: Long = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 9d3e6158266b8bd4f70f8dffe2891d9ea3298d4f..5724ec9d1b4d7aaa2abc5022f5daf8481bc03d0a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -35,7 +35,8 @@ case class IndirectTaskResult[T](blockId: BlockId) extends TaskResult[T] with Se /** A TaskResult that contains the task's return value and accumulator updates. */ private[spark] -class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) +class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long, Any], + var metrics: TaskMetrics) extends TaskResult[T] with Externalizable { def this() = this(null.asInstanceOf[ByteBuffer], null, null) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 17b6d97e90e0a85f1e16090b03d7546d3510dfeb..1cdfed1d7005ef4661faddc883ab966663f869fd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -36,7 +36,8 @@ private[spark] trait TaskScheduler { def start(): Unit // Invoked after system has successfully initialized (typically in spark context). - // Yarn uses this to bootstrap allocation of resources based on preferred locations, wait for slave registerations, etc. + // Yarn uses this to bootstrap allocation of resources based on preferred locations, + // wait for slave registerations, etc. def postStartHook() { } // Disconnect from the cluster. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 3f0ee7a6d48cb5dc4dd02c5a0b1af67d906a56a4..21b2ff1682b78635b7ea9e6cf8a76a5182d753cf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -80,7 +80,7 @@ private[spark] class TaskSetManager( var minShare = 0 var priority = taskSet.priority var stageId = taskSet.stageId - var name = "TaskSet_"+taskSet.stageId.toString + var name = "TaskSet_" + taskSet.stageId.toString var parent: Pool = null val runningTasksSet = new HashSet[Long] diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 0208388e86680754ca01e9a6a3db224a35f73e39..78204103a9bbdac62b26cbbac0972d33ebbceb65 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -120,7 +120,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A sender ! true case DisassociatedEvent(_, address, _) => - addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) + addressToExecutorId.get(address).foreach(removeExecutor(_, + "remote Akka client disassociated")) } 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 33aac52051bfcf278d1a3f3aa3191ae31138cf2e..04f35cca0826294ef3731d5e5523b3daa2b7b1cb 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 @@ -51,8 +51,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() - val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome, - "http://" + sc.ui.appUIAddress) + val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, + sparkHome, "http://" + sc.ui.appUIAddress) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() @@ -84,7 +84,8 @@ private[spark] class SparkDeploySchedulerBackend( } } - override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) { + override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, + memory: Int) { logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format( fullId, hostPort, cores, Utils.megabytesToString(memory))) } 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 c27049bdb520834787b2b30efb5b0cf9d0b916db..4401f6df47421ae02163c0f9f84a3f94f99c3f4b 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 @@ -136,7 +136,8 @@ private[spark] class CoarseMesosSchedulerBackend( // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( - "cd %s*; ./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d" + ("cd %s*; " + + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d") .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index c14cd4755698776cff65a4abe24be5f1a2b26d5c..2d0b25538505c0f0bdb986867768be9ca65a2a7b 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -60,7 +60,8 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial try { for (regCls <- conf.getOption("spark.kryo.registrator")) { logDebug("Running user registrator: " + regCls) - val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] + val reg = Class.forName(regCls, true, classLoader).newInstance() + .asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) } } catch { 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 9a5e3cb77e1d53a501797e4698bf623738fff09c..a38a2b59dbc23b26bad167ba99632f0856fd84fa 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -27,11 +27,12 @@ import org.apache.spark.util.{NextIterator, ByteBufferInputStream} /** * A serializer. Because some serialization libraries are not thread safe, this class is used to - * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual serialization and are - * guaranteed to only be called from one thread at a time. + * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual + * serialization and are guaranteed to only be called from one thread at a time. * * Implementations of this trait should have a zero-arg constructor or a constructor that accepts a - * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes precedence. + * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes + * precedence. */ trait Serializer { def newInstance(): SerializerInstance 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 4fa2ab96d97255c0d8235e0077697a33caa97889..aa62ab5aba1c2a7c3b972c6e9676feacf7191aea 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -76,9 +76,9 @@ object BlockFetcherIterator { import blockManager._ - private var _remoteBytesRead = 0l - private var _remoteFetchTime = 0l - private var _fetchWaitTime = 0l + private var _remoteBytesRead = 0L + private var _remoteFetchTime = 0L + private var _fetchWaitTime = 0L if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") 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 ed53558566edf837bbc3eb3532e3510aedbef80b..542deb98c130456e3bd6a02c5e550293e11329e4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -206,8 +206,9 @@ private[spark] class BlockManager( * message reflecting the current status, *not* the desired storage level in its block info. * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk. * - * droppedMemorySize exists to account for when block is dropped from memory to disk (so it is still valid). - * This ensures that update in master will compensate for the increase in memory on slave. + * droppedMemorySize exists to account for when block is dropped from memory to disk (so it + * is still valid). This ensures that update in master will compensate for the increase in + * memory on slave. */ def reportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L) { val needReregister = !tryToReportBlockStatus(blockId, info, droppedMemorySize) @@ -224,7 +225,8 @@ private[spark] class BlockManager( * which will be true if the block was successfully recorded and false if * the slave needs to re-register. */ - private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L): Boolean = { + private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, + droppedMemorySize: Long = 0L): Boolean = { val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized { info.level match { case null => @@ -282,14 +284,15 @@ private[spark] class BlockManager( // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work if (blockId.isShuffle) { - return diskStore.getBytes(blockId) match { + diskStore.getBytes(blockId) match { case Some(bytes) => Some(bytes) case None => throw new Exception("Block " + blockId + " not found on disk, though it should be") } + } else { + doGetLocal(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] } - doGetLocal(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] } private def doGetLocal(blockId: BlockId, asValues: Boolean): Option[Any] = { @@ -701,7 +704,8 @@ private[spark] class BlockManager( diskStore.putBytes(blockId, bytes, level) } } - val droppedMemorySize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L + val droppedMemorySize = + if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L val blockWasRemoved = memoryStore.remove(blockId) if (!blockWasRemoved) { logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 2c1a4e2f5d3a18ce86faa39831a6403d8ea422e6..893418fb8cad9390d967f95f6fda6bd4ab0e91c0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -61,8 +61,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act override def preStart() { if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { import context.dispatcher - timeoutCheckingTask = context.system.scheduler.schedule( - 0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) + timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, + checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) } super.preStart() } @@ -169,8 +169,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act val toRemove = new mutable.HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { if (info.lastSeenMs < minSeenTime) { - logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + - (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") + logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") toRemove += info.blockManagerId } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 365866d1e33971e96136d43ef97aa2a1128f0209..7cf754fb204c2d1e894debe79c1d382952d90ae0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -57,9 +57,9 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus val diskSpaceUsed = storageStatusList - .flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_ + _) - .getOrElse(0L) + .flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_ + _) + .getOrElse(0L) diskSpaceUsed / 1024 / 1024 } 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 59329361f320b147618cfbd4e5048f801e44a37e..5ded9ab35982085d2957c96db12959a42a1f0577 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala @@ -25,7 +25,8 @@ import org.apache.spark._ import org.apache.spark.network._ private[spark] -class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { +class BlockMessageArray(var blockMessages: Seq[BlockMessage]) + extends Seq[BlockMessage] with Logging { def this(bm: BlockMessage) = this(Array(bm)) @@ -65,7 +66,8 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockM buffer.position(buffer.position() + size) } val finishTime = System.currentTimeMillis - logDebug("Converted block message array from buffer message in " + (finishTime - startTime) / 1000.0 + " s") + logDebug("Converted block message array from buffer message in " + + (finishTime - startTime) / 1000.0 + " s") this.blockMessages = newBlockMessages } 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 1720007e4e70b6aff4685aacc8459a007343afc1..50a0cdb3095cd91adf424f5ad168a18ec75cb4cf 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -25,15 +25,15 @@ private[spark] case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, blocks: Map[BlockId, BlockStatus]) { - def memUsed() = blocks.values.map(_.memSize).reduceOption(_+_).getOrElse(0L) + def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) def memUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_+_).getOrElse(0L) + rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_+_).getOrElse(0L) + def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) def diskUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_+_).getOrElse(0L) + rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) def memRemaining : Long = maxMem - memUsed() @@ -48,8 +48,9 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, extends Ordered[RDDInfo] { override def toString = { import Utils.bytesToString - "RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; DiskSize: %s".format(name, id, - storageLevel.toString, numCachedPartitions, numPartitions, bytesToString(memSize), bytesToString(diskSize)) + ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + + "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, + numPartitions, bytesToString(memSize), bytesToString(diskSize)) } override def compare(that: RDDInfo) = { @@ -64,7 +65,8 @@ object StorageUtils { /* Returns RDD-level information, compiled from a list of StorageStatus objects */ def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus], sc: SparkContext) : Array[RDDInfo] = { - rddInfoFromBlockStatusList(storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus], sc) + rddInfoFromBlockStatusList( + storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus], sc) } /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ @@ -91,7 +93,8 @@ object StorageUtils { sc.persistentRdds.get(rddId).map { r => val rddName = Option(r.name).getOrElse(rddId.toString) val rddStorageLevel = r.getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize) + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, + memSize, diskSize) } }.flatten.toArray 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 5573b3847bcafd3041f18eaf471b6ad374ef26bf..b95c8f43b08f88cf5993fddf1baaca074479a9b7 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -48,14 +48,16 @@ private[spark] object UIUtils { case _ => <li><a href={prependBaseUri("/environment")}>Environment</a></li> } val executors = page match { - case Executors => <li class="active"><a href={prependBaseUri("/executors")}>Executors</a></li> + case Executors => <li class="active"><a href={prependBaseUri("/executors")}>Executors</a> + </li> case _ => <li><a href={prependBaseUri("/executors")}>Executors</a></li> } <html> <head> <meta http-equiv="Content-type" content="text/html; charset=utf-8" /> - <link rel="stylesheet" href={prependBaseUri("/static/bootstrap.min.css")} type="text/css" /> + <link rel="stylesheet" href={prependBaseUri("/static/bootstrap.min.css")} + type="text/css" /> <link rel="stylesheet" href={prependBaseUri("/static/webui.css")} type="text/css" /> <script src={prependBaseUri("/static/sorttable.js")} ></script> <title>{sc.appName} - {title}</title> @@ -63,7 +65,8 @@ private[spark] object UIUtils { <body> <div class="navbar navbar-static-top"> <div class="navbar-inner"> - <a href={prependBaseUri("/")} class="brand"><img src={prependBaseUri("/static/spark-logo-77x50px-hd.png")} /></a> + <a href={prependBaseUri("/")} class="brand"> + <img src={prependBaseUri("/static/spark-logo-77x50px-hd.png")} /></a> <ul class="nav"> {jobs} {storage} @@ -93,7 +96,8 @@ private[spark] object UIUtils { <html> <head> <meta http-equiv="Content-type" content="text/html; charset=utf-8" /> - <link rel="stylesheet" href={prependBaseUri("/static/bootstrap.min.css")} type="text/css" /> + <link rel="stylesheet" href={prependBaseUri("/static/bootstrap.min.css")} + type="text/css" /> <link rel="stylesheet" href={prependBaseUri("/static/webui.css")} type="text/css" /> <script src={prependBaseUri("/static/sorttable.js")} ></script> <title>{title}</title> @@ -103,7 +107,8 @@ private[spark] object UIUtils { <div class="row-fluid"> <div class="span12"> <h3 style="vertical-align: middle; display: inline-block;"> - <img src={prependBaseUri("/static/spark-logo-77x50px-hd.png")} style="margin-right: 15px;" /> + <img src={prependBaseUri("/static/spark-logo-77x50px-hd.png")} + style="margin-right: 15px;" /> {title} </h3> </div> diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 6ba15187d9f637a496513f58bbc0a9b276e60bf8..f913ee461b219cfb68f0d09e53c3b24cad16f05d 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -36,7 +36,8 @@ private[spark] object UIWorkloadGenerator { def main(args: Array[String]) { if (args.length < 2) { - println("usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") + println( + "usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") System.exit(1) } 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 a31a7e1d58374568b84e75572efb76028d3404df..4e41acf0230f9cd5042ce3d0454da6c623de5667 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 @@ -51,9 +51,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = sc.getExecutorStorageStatus - val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_+_) - val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_+_) + val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) + val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", "Active tasks", "Failed tasks", "Complete tasks", "Total tasks", "Task Time", "Shuffle Read", 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 ca5a28625b7de522c1975c8381282e8677481ad7..6289f8744f24090e7c3cce8b8116daca72466eab 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 @@ -43,7 +43,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { } val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) - val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) + val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, + parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) val pools = listener.sc.getAllPools 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 cfeeccda41713c41c8acbf7c7f69dc5291cb38c4..9412a48330d85ca834e5f6d215191bcc4b3b14d5 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 @@ -60,7 +60,10 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis case None => 0 } <tr> - <td><a href={"%s/stages/pool?poolname=%s".format(UIUtils.prependBaseUri(),p.name)}>{p.name}</a></td> + <td> + <a href= + {"%s/stages/pool?poolname=%s".format(UIUtils.prependBaseUri(),p.name)}> + {p.name}</a></td> <td>{p.minShare}</td> <td>{p.weight}</td> <td>{activeStages}</td> 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 cfaf121895ec2c7d54d247330d0b3c88b052020f..08107a3f62232646523ef9acc7024930fe13dff6 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 @@ -64,7 +64,7 @@ private[spark] class StagePage(parent: JobProgressUI) { listener.stageIdToTasksActive(stageId).foreach(activeTime += _.timeRunning(now)) val finishedTasks = listener.stageIdToTaskInfos(stageId).filter(_._1.finished) - + // scalastyle:off val summary = <div> <ul class="unstyled"> @@ -96,7 +96,7 @@ private[spark] class StagePage(parent: JobProgressUI) { } </ul> </div> - + // scalastyle:on val taskHeaders: Seq[String] = Seq("Task Index", "Task ID", "Status", "Locality Level", "Executor", "Launch Time") ++ Seq("Duration", "GC Time", "Result Ser Time") ++ @@ -105,7 +105,8 @@ private[spark] class StagePage(parent: JobProgressUI) { {if (hasBytesSpilled) Seq("Shuffle Spill (Memory)", "Shuffle Spill (Disk)") else Nil} ++ Seq("Errors") - val taskTable = listingTable(taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) + val taskTable = listingTable( + taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) // Excludes tasks which failed and have incomplete metrics val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && (t._2.isDefined)) @@ -117,8 +118,9 @@ private[spark] class StagePage(parent: JobProgressUI) { else { val serializationTimes = validTasks.map{case (info, metrics, exception) => metrics.get.resultSerializationTime.toDouble} - val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes).get.getQuantiles().map( - ms => parent.formatDuration(ms.toLong)) + val serializationQuantiles = + "Result serialization time" +: Distribution(serializationTimes). + get.getQuantiles().map(ms => parent.formatDuration(ms.toLong)) val serviceTimes = validTasks.map{case (info, metrics, exception) => metrics.get.executorRunTime.toDouble} @@ -225,7 +227,8 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") val shuffleReadReadable = maybeShuffleRead.map{Utils.bytesToString(_)}.getOrElse("") - val maybeShuffleWrite = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleBytesWritten} + val maybeShuffleWrite = + metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleBytesWritten} val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") val shuffleWriteReadable = maybeShuffleWrite.map{Utils.bytesToString(_)}.getOrElse("") @@ -236,7 +239,8 @@ private[spark] class StagePage(parent: JobProgressUI) { val maybeMemoryBytesSpilled = metrics.map{m => m.memoryBytesSpilled} val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.map(_.toString).getOrElse("") - val memoryBytesSpilledReadable = maybeMemoryBytesSpilled.map{Utils.bytesToString(_)}.getOrElse("") + val memoryBytesSpilledReadable = maybeMemoryBytesSpilled.map{Utils.bytesToString(_)} + .getOrElse("") val maybeDiskBytesSpilled = metrics.map{m => m.diskBytesSpilled} val diskBytesSpilledSortable = maybeDiskBytesSpilled.map(_.toString).getOrElse("") 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 9ad6de3c6d8de79c758f1d0764b1a171bd56012c..01b647917964277b2e3735d72ae14ed095037410 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 @@ -59,7 +59,8 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr </table> } - private def makeProgressBar(started: Int, completed: Int, failed: String, total: Int): Seq[Node] = { + private def makeProgressBar(started: Int, completed: Int, failed: String, total: Int) + : Seq[Node] = { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) val startWidth = "width: %s%%".format((started.toDouble/total)*100) 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 1df6b87fb0730cf2aa4ecb05632fd1d49ba8d7b3..3eb0f081e4cf67fceb09be663148cb062e617d88 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.Set import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import org.objectweb.asm.Opcodes._ -import java.io.{InputStream, IOException, ByteArrayOutputStream, ByteArrayInputStream, BufferedInputStream} +import java.io.{ByteArrayOutputStream, ByteArrayInputStream} import org.apache.spark.Logging private[spark] object ClosureCleaner extends Logging { @@ -159,8 +159,9 @@ private[spark] object ClosureCleaner extends Logging { // other than to set its fields, so use its constructor val cons = cls.getConstructors()(0) val params = cons.getParameterTypes.map(createNullValue).toArray - if (outer != null) + if (outer != null) { params(0) = outer // First param is always outer object + } return cons.newInstance(params: _*).asInstanceOf[AnyRef] } else { // Use reflection to instantiate object without calling constructor @@ -179,7 +180,8 @@ private[spark] object ClosureCleaner extends Logging { } } -private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { +private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) + extends ClassVisitor(ASM4) { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { new MethodVisitor(ASM4) { @@ -221,11 +223,12 @@ private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisi val argTypes = Type.getArgumentTypes(desc) if (op == INVOKESPECIAL && name == "<init>" && argTypes.length > 0 && argTypes(0).toString.startsWith("L") // is it an object? - && argTypes(0).getInternalName == myName) + && argTypes(0).getInternalName == myName) { output += Class.forName( owner.replace('/', '.'), false, Thread.currentThread.getContextClassLoader) + } } } } diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala index fcc1ca9502aa141217a45d9a0b217697f4c95714..b6a099825f01b87ddc2b7e014f6b286bcf9249c5 100644 --- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala @@ -21,7 +21,10 @@ package org.apache.spark.util * Wrapper around an iterator which calls a completion method after it successfully iterates * through all the elements. */ -private[spark] abstract class CompletionIterator[+A, +I <: Iterator[A]](sub: I) extends Iterator[A]{ +private[spark] +// scalastyle:off +abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterator[A] { +// scalastyle:on def next() = sub.next() def hasNext = { val r = sub.hasNext diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index 33bf3562fe342b70424642f186514be16a7222fd..ab738c4b868fa2f03d712c984d20a24802f43e6d 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -20,7 +20,8 @@ package org.apache.spark.util import java.io.PrintStream /** - * Util for getting some stats from a small sample of numeric values, with some handy summary functions. + * Util for getting some stats from a small sample of numeric values, with some handy + * summary functions. * * Entirely in memory, not intended as a good way to compute stats over large data sets. * @@ -68,10 +69,11 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) object Distribution { def apply(data: Traversable[Double]): Option[Distribution] = { - if (data.size > 0) + if (data.size > 0) { Some(new Distribution(data)) - else + } else { None + } } def showQuantiles(out: PrintStream = System.out, quantiles: Traversable[Double]) { diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index b0febe906ade32e5a9a7a805e47ee4a44816eb0a..3868ab36312a78fc460c031cac52a0b3c2d55d7e 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -67,7 +67,8 @@ private[spark] object MetadataCleanerType extends Enumeration { type MetadataCleanerType = Value - def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = "spark.cleaner.ttl." + which.toString + def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = + "spark.cleaner.ttl." + which.toString } // TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the diff --git a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala index 8b4e7c104cb19424f1868c4e48fc5baa3880c34b..2110b3596ee2ac42c7613ba45d1123916121d2d9 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala @@ -21,7 +21,8 @@ import java.io.{Externalizable, ObjectOutput, ObjectInput} import com.clearspring.analytics.stream.cardinality.{ICardinality, HyperLogLog} /** - * A wrapper around [[com.clearspring.analytics.stream.cardinality.HyperLogLog]] that is serializable. + * A wrapper around [[com.clearspring.analytics.stream.cardinality.HyperLogLog]] that is + * serializable. */ private[spark] class SerializableHyperLogLog(var value: ICardinality) extends Externalizable { 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 3cf94892e9680e14283148b8c70cd2d41048b5c0..5f86795183a9d6512748534a7063e46796d5434c 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -224,24 +224,26 @@ private[spark] object SizeEstimator extends Logging { } private def primitiveSize(cls: Class[_]): Long = { - if (cls == classOf[Byte]) + if (cls == classOf[Byte]) { BYTE_SIZE - else if (cls == classOf[Boolean]) + } else if (cls == classOf[Boolean]) { BOOLEAN_SIZE - else if (cls == classOf[Char]) + } else if (cls == classOf[Char]) { CHAR_SIZE - else if (cls == classOf[Short]) + } else if (cls == classOf[Short]) { SHORT_SIZE - else if (cls == classOf[Int]) + } else if (cls == classOf[Int]) { INT_SIZE - else if (cls == classOf[Long]) + } else if (cls == classOf[Long]) { LONG_SIZE - else if (cls == classOf[Float]) + } else if (cls == classOf[Float]) { FLOAT_SIZE - else if (cls == classOf[Double]) + } else if (cls == classOf[Double]) { DOUBLE_SIZE - else throw new IllegalArgumentException( + } else { + throw new IllegalArgumentException( "Non-primitive class " + cls + " passed to primitiveSize()") + } } /** 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 020d5edba9530d4eefee2e80579b795b0af73860..5b0d2c36510b8a6ac27a66f006ee32d9ebd369a4 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -20,7 +20,8 @@ package org.apache.spark.util /** * A class for tracking the statistics of a set of numbers (count, mean and variance) in a * numerically robust way. Includes support for merging two StatCounters. Based on - * [[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance Welford and Chan's algorithms for running variance]]. + * [[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance + * Welford and Chan's algorithms for running variance]]. * * @constructor Initialize the StatCounter with the given values. */ @@ -70,7 +71,7 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { m2 += other.m2 + (delta * delta * n * other.n) / (n + other.n) n += other.n } - this + this } } @@ -91,10 +92,11 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { /** Return the variance of the values. */ def variance: Double = { - if (n == 0) + if (n == 0) { Double.NaN - else + } else { m2 / n + } } /** @@ -102,10 +104,11 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { * by N-1 instead of N. */ def sampleVariance: Double = { - if (n <= 1) + if (n <= 1) { Double.NaN - else + } else { m2 / (n - 1) + } } /** Return the standard deviation of the values. */ 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 861ad62f9f59fddb0645c7c6c5cfd684df89e3dd..c201d0a33f4b6544a87e50d9b4a674f7a698ce79 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -86,7 +86,8 @@ private[spark] object Utils extends Logging { } /** Serialize via nested stream using specific serializer */ - def serializeViaNestedStream(os: OutputStream, ser: SerializerInstance)(f: SerializationStream => Unit) = { + def serializeViaNestedStream(os: OutputStream, ser: SerializerInstance)( + f: SerializationStream => Unit) = { val osWrapper = ser.serializeStream(new OutputStream { def write(b: Int) = os.write(b) @@ -100,7 +101,8 @@ private[spark] object Utils extends Logging { } /** Deserialize via nested stream using specific serializer */ - def deserializeViaNestedStream(is: InputStream, ser: SerializerInstance)(f: DeserializationStream => Unit) = { + def deserializeViaNestedStream(is: InputStream, ser: SerializerInstance)( + f: DeserializationStream => Unit) = { val isWrapper = ser.deserializeStream(new InputStream { def read(): Int = is.read() 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 83fa0bf1e583f579bf66632494c34d509446a292..96da93d8545770c5c7d1f85a9a804f3ee281ecf7 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -26,24 +26,27 @@ class Vector(val elements: Array[Double]) extends Serializable { def apply(index: Int) = elements(index) def + (other: Vector): Vector = { - if (length != other.length) + if (length != other.length) { throw new IllegalArgumentException("Vectors of different length") + } Vector(length, i => this(i) + other(i)) } def add(other: Vector) = this + other def - (other: Vector): Vector = { - if (length != other.length) + if (length != other.length) { throw new IllegalArgumentException("Vectors of different length") + } Vector(length, i => this(i) - other(i)) } def subtract(other: Vector) = this - other def dot(other: Vector): Double = { - if (length != other.length) + if (length != other.length) { throw new IllegalArgumentException("Vectors of different length") + } var ans = 0.0 var i = 0 while (i < length) { @@ -60,10 +63,12 @@ class Vector(val elements: Array[Double]) extends Serializable { * @return */ def plusDot(plus: Vector, other: Vector): Double = { - if (length != other.length) + if (length != other.length) { throw new IllegalArgumentException("Vectors of different length") - if (length != plus.length) + } + if (length != plus.length) { throw new IllegalArgumentException("Vectors of different length") + } var ans = 0.0 var i = 0 while (i < length) { @@ -74,8 +79,9 @@ class Vector(val elements: Array[Double]) extends Serializable { } def += (other: Vector): Vector = { - if (length != other.length) + if (length != other.length) { throw new IllegalArgumentException("Vectors of different length") + } var i = 0 while (i < length) { elements(i) += other(i) @@ -131,7 +137,8 @@ object Vector { * 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()) = Vector(length, _ => random.nextDouble()) + def random(length: Int, random: Random = new XORShiftRandom()) = + Vector(length, _ => random.nextDouble()) class Multiplier(num: Double) { def * (vec: Vector) = vec * num diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 856eb772a10840c035db3cfe069239519dab51d0..c9cf51284387529ae2f820f5ed59a077cbbc88d2 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -123,7 +123,7 @@ class BitSet(numBits: Int) extends Serializable { override def hasNext: Boolean = ind >= 0 override def next() = { val tmp = ind - ind = nextSetBit(ind+1) + ind = nextSetBit(ind + 1) tmp } } 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 7eb300d46e6e2f6ca7323380675a434d5e96e65e..59ba1e457ce115319bec8a99786022c1b4a3f5c0 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 @@ -280,7 +280,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( /** * Select a key with the minimum hash, then combine all values with the same key from all - * input streams + * input streams. */ override def next(): (K, C) = { // Select a key from the StreamBuffer that holds the lowest key hash 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 5ded5d0b6da8466a7560295fc12ff930a7f8cfae..148c12e64d2ce5f205d8c5f7c7229d3b7e1b3486 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 @@ -187,7 +187,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( override def hasNext: Boolean = pos != INVALID_POS override def next(): T = { val tmp = getValue(pos) - pos = nextPos(pos+1) + pos = nextPos(pos + 1) tmp } } diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index 88f1cef89b318faf126626efbacd9b4594b03cf0..c2d84a8e0861ec4d4e3084797966bb4b66ab16a1 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -19,18 +19,21 @@ package org.apache.spark.streaming.examples import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ - +// scalastyle:off /** - * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every second. + * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every + * second. * Usage: StatefulNetworkWordCount <master> <hostname> <port> * <master> is the Spark master URL. In local mode, <master> should be 'local[n]' with n > 1. - * <hostname> and <port> describe the TCP server that Spark Streaming would connect to receive data. + * <hostname> and <port> describe the TCP server that Spark Streaming would connect to receive + * data. * * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example * `$ ./bin/run-example org.apache.spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ +// scalastyle:on object StatefulNetworkWordCount { def main(args: Array[String]) { if (args.length < 3) { @@ -50,8 +53,8 @@ object StatefulNetworkWordCount { } // Create the context with a 1 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", + Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index a0094d460feecf89bb7050107f1d3475dfcf17e3..c6215fd0d7561818338b17c8ae3b4ac1177122a8 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -23,20 +23,24 @@ import com.twitter.algebird.HyperLogLog._ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.twitter._ - +// scalastyle:off /** * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute * a windowed and global estimate of the unique user IDs occurring in a Twitter stream. * <p> * <p> - * This <a href="http://highlyscalable.wordpress.com/2012/05/01/probabilistic-structures-web-analytics-data-mining/"> + * This <a href= "http://highlyscalable.wordpress.com/2012/05/01/probabilistic-structures-web-analytics-data + * -mining/"> * blog post</a> and this - * <a href="http://highscalability.com/blog/2012/4/5/big-data-counting-how-to-count-a-billion-distinct-objects-us.html">blog post</a> - * have good overviews of HyperLogLog (HLL). HLL is a memory-efficient datastructure for estimating - * the cardinality of a data stream, i.e. the number of unique elements. + * <a href= "http://highscalability.com/blog/2012/4/5/big-data-counting-how-to-count-a-billion-distinct-objects-us.html"> + * blog post</a> + * have good overviews of HyperLogLog (HLL). HLL is a memory-efficient datastructure for + * estimating the cardinality of a data stream, i.e. the number of unique elements. * <p><p> - * Algebird's implementation is a monoid, so we can succinctly merge two HLL instances in the reduce operation. + * Algebird's implementation is a monoid, so we can succinctly merge two HLL instances in the + * reduce operation. */ +// scalastyle:on object TwitterAlgebirdHLL { def main(args: Array[String]) { if (args.length < 1) { @@ -82,7 +86,8 @@ object TwitterAlgebirdHLL { userSet ++= partial println("Exact distinct users this batch: %d".format(partial.size)) println("Exact distinct users overall: %d".format(userSet.size)) - println("Error rate: %2.5f%%".format(((globalHll.estimatedSize / userSet.size.toDouble) - 1) * 100)) + println("Error rate: %2.5f%%".format(((globalHll.estimatedSize / userSet.size.toDouble) - 1 + ) * 100)) } }) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala index a2600989ca1a62bda36526215829e21cbab7460d..0ac46c31c24c862d058f99e684c1d0a9868bd099 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -36,6 +36,7 @@ object PageView extends Serializable { } } +// scalastyle:off /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: @@ -44,7 +45,8 @@ object PageView extends Serializable { * * When running this, you may want to set the root logging level to ERROR in * conf/log4j.properties to reduce the verbosity of the output. - * */ + */ +// scalastyle:on object PageViewGenerator { val pages = Map("http://foo.com/" -> .7, "http://foo.com/news" -> 0.2, diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index bb44bc3d06ef3918ff8b80352ae6e14b62c7c646..2b130fb30eaaaabc80726c4fb36925662e60b8b4 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.examples.StreamingExamples - +// scalastyle:off /** Analyses a streaming dataset of web page views. This class demonstrates several types of * operators available in Spark streaming. * @@ -29,6 +29,7 @@ import org.apache.spark.streaming.examples.StreamingExamples * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10 * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 */ +// scalastyle:on object PageViewStream { def main(args: Array[String]) { if (args.length != 3) { diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index a2cd49c573fb539e281db3d532773e0cb3cbb103..c2d9dcbfaac7acab3a4d042aa79af2301368794c 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -37,7 +37,8 @@ import org.apache.spark.streaming.dstream._ /** * Input stream that pulls messages from a Kafka Broker. * - * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param kafkaParams Map of kafka configuration paramaters. + * See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param storageLevel RDD storage level. @@ -134,12 +135,15 @@ class KafkaReceiver[ } } - // It is our responsibility to delete the consumer group when specifying autooffset.reset. This is because - // Kafka 0.7.2 only honors this param when the group is not in zookeeper. + // It is our responsibility to delete the consumer group when specifying autooffset.reset. This + // is because Kafka 0.7.2 only honors this param when the group is not in zookeeper. // - // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied from Kafkas' - // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest'/'largest': + // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied + // from Kafkas' ConsoleConsumer. See code related to 'autooffset.reset' when it is set to + // 'smallest'/'largest': + // scalastyle:off // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala + // scalastyle:on private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { try { val dir = "/consumers/" + groupId diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 15a2daa008ad489c3d10c669649ccc8dfe62a378..5472d0cd04a94edd1c0a286380f2a71f06027612 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -113,7 +113,8 @@ object KafkaUtils { ): JavaPairDStream[String, String] = { implicit val cmt: ClassTag[String] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), + storageLevel) } /** diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala index 960c6a389ef0f1a3902fed497dd3c45096883b0f..6acba25f44c0a5278cff9d1519d50f54455347df 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -34,8 +34,8 @@ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, bytesToObjects: Seq[ByteString] => Iterator[T]) extends Actor with Receiver with Logging { - override def preStart() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self), - Connect(publisherUrl), subscribe) + override def preStart() = ZeroMQExtension(context.system) + .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) def receive: Receive = { diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala index b47d786986a917e55bfbde287a7fe0a6554f1d1f..c989ec0f27465ceba9b409e1ef1b1f7be12fb6f4 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala @@ -59,10 +59,10 @@ object ZeroMQUtils { * @param jssc JavaStreamingContext object * @param publisherUrl Url of remote ZeroMQ publisher * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each + * frame has sequence of byte thus it needs the converter(which might be + * deserializer of bytes) to translate from sequence of sequence of bytes, + * where sequence refer to a frame and sub sequence refer to its payload. * @param storageLevel Storage level to use for storing the received objects */ def createStream[T]( @@ -84,10 +84,10 @@ object ZeroMQUtils { * @param jssc JavaStreamingContext object * @param publisherUrl Url of remote zeromq publisher * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each + * frame has sequence of byte thus it needs the converter(which might be + * deserializer of bytes) to translate from sequence of sequence of bytes, + * where sequence refer to a frame and sub sequence refer to its payload. * @param storageLevel RDD storage level. */ def createStream[T]( @@ -108,10 +108,11 @@ object ZeroMQUtils { * @param jssc JavaStreamingContext object * @param publisherUrl Url of remote zeromq publisher * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each + * frame has sequence of byte thus it needs the converter(which might + * be deserializer of bytes) to translate from sequence of sequence of + * bytes, where sequence refer to a frame and sub sequence refer to its + * payload. */ def createStream[T]( jssc: JavaStreamingContext, diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index fe03ae4a629b9fef4fffe2a9bf41a3a29e739c6b..799a9dd1ee5b59630b5f7101c9ecf42d4d974563 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -66,7 +66,8 @@ class EdgeRDD[@specialized ED: ClassTag]( this } - private[graphx] def mapEdgePartitions[ED2: ClassTag](f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) + private[graphx] def mapEdgePartitions[ED2: ClassTag]( + f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]) : EdgeRDD[ED2] = { new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => val (pid, ep) = iter.next() @@ -97,8 +98,8 @@ class EdgeRDD[@specialized ED: ClassTag]( * * @param other the EdgeRDD to join with * @param f the join function applied to corresponding values of `this` and `other` - * @return a new EdgeRDD containing only edges that appear in both `this` and `other`, with values - * supplied by `f` + * @return a new EdgeRDD containing only edges that appear in both `this` and `other`, + * with values supplied by `f` */ def innerJoin[ED2: ClassTag, ED3: ClassTag] (other: EdgeRDD[ED2]) 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 eea95d38d50166f7c718c7aa4206b0a1e1aec9d2..65a1a8c68f6d2c48b82846f305b453a2acf4598d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -171,8 +171,9 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab : Graph[VD, ED2] /** - * Transforms each edge attribute using the map function, passing it the adjacent vertex attributes - * as well. If adjacent vertex values are not required, consider using `mapEdges` instead. + * Transforms each edge attribute using the map function, passing it the adjacent vertex + * attributes as well. If adjacent vertex values are not required, + * consider using `mapEdges` instead. * * @note This does not change the structure of the * graph or modify the values of this graph. As a consequence @@ -280,13 +281,13 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * be commutative and associative and is used to combine the output * of the map phase * - * @param activeSetOpt optionally, a set of "active" vertices and a direction of edges to consider - * when running `mapFunc`. If the direction is `In`, `mapFunc` will only be run on edges with - * destination in the active set. If the direction is `Out`, `mapFunc` will only be run on edges - * originating from vertices in the active set. If the direction is `Either`, `mapFunc` will be - * run on edges with *either* vertex in the active set. If the direction is `Both`, `mapFunc` will - * be run on edges with *both* vertices in the active set. The active set must have the same index - * as the graph's vertices. + * @param activeSetOpt optionally, a set of "active" vertices and a direction of edges to + * consider when running `mapFunc`. If the direction is `In`, `mapFunc` will only be run on + * edges with destination in the active set. If the direction is `Out`, + * `mapFunc` will only be run on edges originating from vertices in the active set. If the + * direction is `Either`, `mapFunc` will be run on edges with *either* vertex in the active set + * . If the direction is `Both`, `mapFunc` will be run on edges with *both* vertices in the + * active set. The active set must have the same index as the graph's vertices. * * @example We can use this function to compute the in-degree of each * vertex diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 929915362c1c9f7ce5c4cd8d5de0e97dcb7e2650..0470d74cf9efe36ae4a8934fbeb79219009d82d9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -57,8 +57,9 @@ object PartitionStrategy { * </pre> * * The edge denoted by `E` connects `v11` with `v1` and is assigned to processor `P6`. To get the - * processor number we divide the matrix into `sqrt(numParts)` by `sqrt(numParts)` blocks. Notice - * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3, P6)` or the last + * processor number we divide the matrix into `sqrt(numParts)` by `sqrt(numParts)` blocks. Notice + * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3, + * P6)` or the last * row of blocks `(P6, P7, P8)`. As a consequence we can guarantee that `v11` will need to be * replicated to at most `2 * sqrt(numParts)` machines. * @@ -66,11 +67,12 @@ object PartitionStrategy { * balance. To improve balance we first multiply each vertex id by a large prime to shuffle the * vertex locations. * - * One of the limitations of this approach is that the number of machines must either be a perfect - * square. We partially address this limitation by computing the machine assignment to the next + * One of the limitations of this approach is that the number of machines must either be a + * perfect square. We partially address this limitation by computing the machine assignment to + * the next * largest perfect square and then mapping back down to the actual number of machines. - * Unfortunately, this can also lead to work imbalance and so it is suggested that a perfect square - * is used. + * Unfortunately, this can also lead to work imbalance and so it is suggested that a perfect + * square is used. */ case object EdgePartition2D extends PartitionStrategy { override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index edd59bcf329438022553d2f07118d80597bbc0e6..d6788d4d4b9fd57186097789ea0accdae99eb8fd 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -59,7 +59,8 @@ class VertexRDD[@specialized VD: ClassTag]( /** * Construct a new VertexRDD that is indexed by only the visible vertices. The resulting - * VertexRDD will be based on a different index and can no longer be quickly joined with this RDD. + * VertexRDD will be based on a different index and can no longer be quickly joined with this + * RDD. */ def reindex(): VertexRDD[VD] = new VertexRDD(partitionsRDD.map(_.reindex())) @@ -101,7 +102,8 @@ class VertexRDD[@specialized VD: ClassTag]( /** * Applies a function to each `VertexPartition` of this RDD and returns a new VertexRDD. */ - private[graphx] def mapVertexPartitions[VD2: ClassTag](f: VertexPartition[VD] => VertexPartition[VD2]) + private[graphx] def mapVertexPartitions[VD2: ClassTag]( + f: VertexPartition[VD] => VertexPartition[VD2]) : VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) new VertexRDD(newPartitionsRDD) @@ -159,8 +161,9 @@ class VertexRDD[@specialized VD: ClassTag]( } /** - * Left joins this RDD with another VertexRDD with the same index. This function will fail if both - * VertexRDDs do not share the same index. The resulting vertex set contains an entry for each + * Left joins this RDD with another VertexRDD with the same index. This function will fail if + * both VertexRDDs do not share the same index. The resulting vertex set contains an entry for + * each * vertex in `this`. If `other` is missing any vertex in this VertexRDD, `f` is passed `None`. * * @tparam VD2 the attribute type of the other VertexRDD @@ -187,8 +190,8 @@ class VertexRDD[@specialized VD: ClassTag]( * Left joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is * backed by a VertexRDD with the same index then the efficient [[leftZipJoin]] implementation is * used. The resulting VertexRDD contains an entry for each vertex in `this`. If `other` is - * missing any vertex in this VertexRDD, `f` is passed `None`. If there are duplicates, the vertex - * is picked arbitrarily. + * missing any vertex in this VertexRDD, `f` is passed `None`. If there are duplicates, + * the vertex is picked arbitrarily. * * @tparam VD2 the attribute type of the other VertexRDD * @tparam VD3 the attribute type of the resulting VertexRDD @@ -238,14 +241,14 @@ class VertexRDD[@specialized VD: ClassTag]( /** * Inner joins this VertexRDD with an RDD containing vertex attribute pairs. If the other RDD is - * backed by a VertexRDD with the same index then the efficient [[innerZipJoin]] implementation is - * used. + * backed by a VertexRDD with the same index then the efficient [[innerZipJoin]] implementation + * is used. * * @param other an RDD containing vertices to join. If there are multiple entries for the same * vertex, one is picked arbitrarily. Use [[aggregateUsingIndex]] to merge multiple entries. * @param f the join function applied to corresponding values of `this` and `other` - * @return a VertexRDD co-indexed with `this`, containing only vertices that appear in both `this` - * and `other`, with values supplied by `f` + * @return a VertexRDD co-indexed with `this`, containing only vertices that appear in both + * `this` and `other`, with values supplied by `f` */ def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexId, U)]) (f: (VertexId, VD, U) => VD2): VertexRDD[VD2] = { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala index f914e0565ca2149586cb8b366e2709854c643cf4..24699dfdd38b05eeb1d645e1e79b51d1de4ba275 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -82,7 +82,7 @@ object Analytics extends Logging { val pr = graph.pageRank(tol).vertices.cache() - println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_+_)) + println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_ + _)) if (!outFname.isEmpty) { logWarning("Saving pageranks of pages to " + outFname) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 7677641bfede62645adcd552b56c1eb0fe048c22..f841846c0e5100ee38f09c9ae1342147ab35b1bb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -37,11 +37,7 @@ object GraphGenerators { val RMATa = 0.45 val RMATb = 0.15 - val RMATc = 0.15 val RMATd = 0.25 - - // Right now it just generates a bunch of edges where - // the edge data is the weight (default 1) /** * Generate a graph whose vertex out degree is log normal. */ @@ -59,15 +55,20 @@ object GraphGenerators { Graph(vertices, edges, 0) } + // Right now it just generates a bunch of edges where + // the edge data is the weight (default 1) + val RMATc = 0.15 + def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int): Array[Edge[Int]] = { val rand = new Random() Array.fill(maxVertexId) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } } /** - * Randomly samples from a log normal distribution whose corresponding normal distribution has the - * the given mean and standard deviation. It uses the formula `X = exp(m+s*Z)` where `m`, `s` are - * the mean, standard deviation of the lognormal distribution and `Z ~ N(0, 1)`. In this function, + * Randomly samples from a log normal distribution whose corresponding normal distribution has + * the given mean and standard deviation. It uses the formula `X = exp(m+s*Z)` where `m`, + * `s` are the mean, standard deviation of the lognormal distribution and + * `Z ~ N(0, 1)`. In this function, * `m = e^(mu+sigma^2/2)` and `s = sqrt[(e^(sigma^2) - 1)(e^(2*mu+sigma^2))]`. * * @param mu the mean of the normal distribution @@ -76,7 +77,7 @@ object GraphGenerators { */ private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = { val rand = new Random() - val m = math.exp(mu+(sigma*sigma)/2.0) + val m = math.exp(mu + (sigma * sigma) / 2.0) val s = math.sqrt((math.exp(sigma*sigma) - 1) * math.exp(2*mu + sigma*sigma)) // Z ~ N(0, 1) var X: Double = maxVal @@ -169,9 +170,9 @@ object GraphGenerators { val newT = math.round(t.toFloat/2.0).toInt pickQuadrant(RMATa, RMATb, RMATc, RMATd) match { case 0 => chooseCell(x, y, newT) - case 1 => chooseCell(x+newT, y, newT) - case 2 => chooseCell(x, y+newT, newT) - case 3 => chooseCell(x+newT, y+newT, newT) + case 1 => chooseCell(x + newT, y, newT) + case 2 => chooseCell(x, y + newT, newT) + case 3 => chooseCell(x + newT, y + newT, newT) } } } @@ -179,8 +180,8 @@ object GraphGenerators { // TODO(crankshaw) turn result into an enum (or case class for pattern matching} private def pickQuadrant(a: Double, b: Double, c: Double, d: Double): Int = { if (a + b + c + d != 1.0) { - throw new IllegalArgumentException( - "R-MAT probability parameters sum to " + (a+b+c+d) + ", should sum to 1.0") + throw new IllegalArgumentException("R-MAT probability parameters sum to " + (a + b + c + d) + + ", should sum to 1.0") } val rand = new Random() val result = rand.nextDouble() @@ -212,8 +213,8 @@ object GraphGenerators { sc.parallelize(0 until rows).flatMap( r => (0 until cols).map( c => (sub2ind(r,c), (r,c)) ) ) val edges: RDD[Edge[Double]] = vertices.flatMap{ case (vid, (r,c)) => - (if (r+1 < rows) { Seq( (sub2ind(r, c), sub2ind(r+1, c))) } else { Seq.empty }) ++ - (if (c+1 < cols) { Seq( (sub2ind(r, c), sub2ind(r, c+1))) } else { Seq.empty }) + (if (r + 1 < rows) { Seq( (sub2ind(r, c), sub2ind(r + 1, c))) } else { Seq.empty }) ++ + (if (c + 1 < cols) { Seq( (sub2ind(r, c), sub2ind(r, c + 1))) } else { Seq.empty }) }.map{ case (src, dst) => Edge(src, dst, 1.0) } Graph(vertices, edges) } // end of gridGraph diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index efc0eb935376bf0ffa5686e19240239679ba55e1..efe99a31beac4f6e156278054e9d5751e7308cca 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -106,7 +106,8 @@ class PythonMLLibAPI extends Serializable { bytes } - private def trainRegressionModel(trainFunc: (RDD[LabeledPoint], Array[Double]) => GeneralizedLinearModel, + private def trainRegressionModel( + trainFunc: (RDD[LabeledPoint], Array[Double]) => GeneralizedLinearModel, dataBytesJRDD: JavaRDD[Array[Byte]], initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { val data = dataBytesJRDD.rdd.map(xBytes => { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index e476b534503dc8ba2790356ef1a49bbb4efd88d3..8803c4c1a07be815eba06814fe61fb40166f8181 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -105,7 +105,7 @@ object SVD { cols.flatMap{ case (colind1, mval1) => cols.map{ case (colind2, mval2) => ((colind1, colind2), mval1*mval2) } } - }.reduceByKey(_+_) + }.reduceByKey(_ + _) // Construct jblas A^T A locally val ata = DoubleMatrix.zeros(n, n) @@ -145,10 +145,10 @@ object SVD { // Multiply A by VS^-1 val aCols = data.map(entry => (entry.j, (entry.i, entry.mval))) val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) - val retUdata = aCols.join(bRows).map( {case (key, ( (rowInd, rowVal), (colInd, colVal)) ) - => ((rowInd, colInd), rowVal*colVal)}).reduceByKey(_+_) + val retUdata = aCols.join(bRows).map( {case (key, ( (rowInd, rowVal), (colInd, colVal))) + => ((rowInd, colInd), rowVal*colVal)}).reduceByKey(_ + _) .map{ case ((row, col), mval) => MatrixEntry(row, col, mval)} - val retU = SparseMatrix(retUdata, m, sigma.length) + val retU = SparseMatrix(retUdata, m, sigma.length) MatrixSVD(retU, retS, retV) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index b77364e08ddacffdad88163438ea996b9f79d7e7..cd8013473798dc356037f5154d1e8a8947df3b84 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -142,7 +142,7 @@ object GradientDescent extends Logging { var regVal = 0.0 for (i <- 1 to numIterations) { - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i).map { case (y, features) => val featuresCol = new DoubleMatrix(features.length, 1, features:_*) val (grad, loss) = gradient.compute(featuresCol, y, weights) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index c5f64b13500ce675e1de46a0594ac4d762bc80bd..a990e0fb01c22dd96170c8511eb2995c86f21ceb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -84,8 +84,9 @@ case class Rating(val user: Int, val product: Int, val rating: Double) * [[http://research.yahoo.com/pub/2433]], adapted for the blocked approach used here. * * Essentially instead of finding the low-rank approximations to the rating matrix `R`, - * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if r > 0 - * and 0 if r = 0. The ratings then act as 'confidence' values related to strength of indicated user + * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if + * r > 0 and 0 if r = 0. The ratings then act as 'confidence' values related to strength of + * indicated user * preferences rather than explicit ratings given to items. */ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var lambda: Double, @@ -152,8 +153,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) - // Initialize user and product factors randomly, but use a deterministic seed for each partition - // so that fault recovery works + // Initialize user and product factors randomly, but use a deterministic seed for each + // partition so that fault recovery works val seedGen = new Random() val seed1 = seedGen.nextInt() val seed2 = seedGen.nextInt() @@ -268,7 +269,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray // Sort them by product ID val ordering = new Ordering[(Int, ArrayBuffer[Rating])] { - def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = a._1 - b._1 + def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = + a._1 - b._1 } Sorting.quickSort(groupedRatings)(ordering) // Translate the user IDs to indices based on userIdToPos @@ -369,7 +371,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val tempXtX = DoubleMatrix.zeros(triangleSize) val fullXtX = DoubleMatrix.zeros(rank, rank) - // Compute the XtX and Xy values for each user by adding products it rated in each product block + // Compute the XtX and Xy values for each user by adding products it rated in each product + // block for (productBlock <- 0 until numBlocks) { for (p <- 0 until blockFactors(productBlock).length) { val x = new DoubleMatrix(blockFactors(productBlock)(p)) @@ -544,9 +547,8 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) */ - def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, alpha: Double) - : MatrixFactorizationModel = - { + def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, + alpha: Double): MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, lambda, -1, alpha) } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 11a937e011718d836227bc04ad3dec209a6690ba..bb79f0cd73e2bbb5e9bdf095756e53a15bd9b4c9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -21,6 +21,8 @@ import Keys._ import sbtassembly.Plugin._ import AssemblyKeys._ import scala.util.Properties +import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} + // For Sonatype publishing //import com.jsuereth.pgp.sbtplugin.PgpKeys._ @@ -231,7 +233,7 @@ object SparkBuild extends Build { publishMavenStyle in MavenCompile := true, publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal), publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn - ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings + ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings val slf4jVersion = "1.7.2" diff --git a/project/build.properties b/project/build.properties index 839f5fbb0c34721bdb79c07ce77a0314ad5fc4c9..4b52bb928a66e12a1095f70f5a7586c44e8186da 100644 --- a/project/build.properties +++ b/project/build.properties @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. # -sbt.version=0.12.4 +sbt.version=0.13.1 diff --git a/project/plugins.sbt b/project/plugins.sbt index 4ba0e4280a9deb4ac9259f1af12daf33cca2c7dd..914f2e05a402a02962ce0aa9a7bb862973d0d522 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,10 +1,10 @@ +scalaVersion := "2.10.3" + resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns) resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" -resolvers += "Spray Repository" at "http://repo.spray.cc/" - -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.2") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.10.2") addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") @@ -15,4 +15,7 @@ addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.5.1") //addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6") -addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.3") +addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") + +addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.4.0") + diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala index 4853be2617684c70943e95ec57a32d423f9e4d89..0392a6051fc3bf6ad4b00552367d787a6cf6c9b3 100644 --- a/project/project/SparkPluginBuild.scala +++ b/project/project/SparkPluginBuild.scala @@ -20,5 +20,5 @@ import sbt._ object SparkPluginDef extends Build { lazy val root = Project("plugins", file(".")) dependsOn(junitXmlListener) /* This is not published in a Maven repository, so we get it from GitHub directly */ - lazy val junitXmlListener = uri("https://github.com/ijuma/junit_xml_listener.git#fe434773255b451a38e8d889536ebc260f4225ce") + lazy val junitXmlListener = uri("git://github.com/chenkelmann/junit_xml_listener.git#3f8029fbfda54dc7a68b1afd2f885935e1090016") } 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 3e171849e34941061ee63b01a8157edfe0bf4b25..e3bcf7f30ac8d93fd8a57ff3d59a28fd1c3db146 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -39,20 +39,22 @@ extends ClassLoader(parent) { // Hadoop FileSystem object for our URI, if it isn't using HTTP var fileSystem: FileSystem = { - if (uri.getScheme() == "http") + if (uri.getScheme() == "http") { null - else + } else { FileSystem.get(uri, new Configuration()) + } } override def findClass(name: String): Class[_] = { try { val pathInDirectory = name.replace('.', '/') + ".class" val inputStream = { - if (fileSystem != null) + if (fileSystem != null) { fileSystem.open(new Path(directory, pathInDirectory)) - else + } else { new URL(classUri + "/" + urlEncode(pathInDirectory)).openStream() + } } val bytes = readAndTransformClass(name, inputStream) inputStream.close() @@ -81,10 +83,11 @@ extends ClassLoader(parent) { var done = false while (!done) { val num = in.read(bytes) - if (num >= 0) + if (num >= 0) { bos.write(bytes, 0, num) - else + } else { done = true + } } return bos.toByteArray } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala index b2e1df173eb2642fbc28694e43676cb07c499105..dcc139544e2c2695abe552e8147755f2a616d4ef 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -1,3 +1,5 @@ +// scalastyle:off + /* NSC -- new Scala compiler * Copyright 2005-2013 LAMP/EPFL * @author Paul Phillips 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 87d94d51be199a0c9e6b27b856695715a9096d65..bc25b50a4efc7e091f35120cccd5a1e79a95eae3 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1,3 +1,5 @@ +// scalastyle:off + /* NSC -- new Scala compiler * Copyright 2005-2013 LAMP/EPFL * @author Alexander Spoon 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 efe45240e9b2e119eff7beec07e462d9a5c953d1..3ebf288130fb63e1fba921fdb6e8dba063046263 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -1,3 +1,5 @@ +// scalastyle:off + /* NSC -- new Scala compiler * Copyright 2005-2013 LAMP/EPFL * @author Paul Phillips diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 59fdb0b37a7663de7469767cb791322c0a31bbae..1d73d0b6993a8176d1dee9d6f074aa46706aea96 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -1,3 +1,5 @@ +// scalastyle:off + /* NSC -- new Scala compiler * Copyright 2005-2013 LAMP/EPFL * @author Martin Odersky 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 64084209e80d202c15f6948d066f3cee69178e8e..8f61a5e835044a4f49438b6069bcebaea0b99a15 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -1,3 +1,5 @@ +// scalastyle:off + /* NSC -- new Scala compiler * Copyright 2005-2013 LAMP/EPFL * @author Paul Phillips diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala index 8865f82bc00551efdb4f4cd4debb5b7454fb917c..3159b70008ae35a08911ab02b2a0de33701b24e0 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala @@ -1,3 +1,5 @@ +// scalastyle:off + /* NSC -- new Scala compiler * Copyright 2005-2013 LAMP/EPFL * @author Paul Phillips diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala index 60a4d7841eb7f89a664f22b0020f0d800f188670..946e71039088d056bb36ab6afe18c929a26881d6 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala @@ -1,3 +1,5 @@ +// scalastyle:off + /* NSC -- new Scala compiler * Copyright 2005-2013 LAMP/EPFL * @author Stepan Koltsov diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala index 382f8360a7308e15097530e532b299613f4c308a..13cd2b7fa56c7570016b169a5810eeaf449da57a 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala @@ -1,3 +1,5 @@ +// scalastyle:off + /* NSC -- new Scala compiler * Copyright 2005-2013 LAMP/EPFL * @author Martin Odersky diff --git a/scalastyle-config.xml b/scalastyle-config.xml new file mode 100644 index 0000000000000000000000000000000000000000..752723267633e209e3651fcba3363a5b63b457f6 --- /dev/null +++ b/scalastyle-config.xml @@ -0,0 +1,126 @@ +<!-- If you wish to turn off checking for a section of code, you can put a comment in the source before and after the section, with the following syntax: --> +<!-- // scalastyle:off --> +<!-- ... --> +<!-- // naughty stuff --> +<!-- ... --> +<!-- // scalastyle:on --> + +<scalastyle> + <name>Scalastyle standard configuration</name> + <check level="error" class="org.scalastyle.file.FileTabChecker" enabled="true"></check> + <!-- <check level="error" class="org.scalastyle.file.FileLengthChecker" enabled="true"> --> + <!-- <parameters> --> + <!-- <parameter name="maxFileLength"><![CDATA[800]]></parameter> --> + <!-- </parameters> --> + <!-- </check> --> + <check level="error" class="org.scalastyle.file.HeaderMatchesChecker" enabled="true"> + <parameters> + <parameter name="header"><![CDATA[/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */]]></parameter> + </parameters> + </check> + <check level="error" class="org.scalastyle.scalariform.SpacesAfterPlusChecker" enabled="true"></check> + <check level="error" class="org.scalastyle.file.WhitespaceEndOfLineChecker" enabled="false"></check> + <check level="error" class="org.scalastyle.scalariform.SpacesBeforePlusChecker" enabled="true"></check> + <check level="error" class="org.scalastyle.file.FileLineLengthChecker" enabled="true"> + <parameters> + <parameter name="maxLineLength"><![CDATA[100]]></parameter> + <parameter name="tabSize"><![CDATA[2]]></parameter> + <parameter name="ignoreImports">true</parameter> + </parameters> + </check> + <check level="error" class="org.scalastyle.scalariform.ClassNamesChecker" enabled="true"> + <parameters> + <parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter> + </parameters> + </check> + <check level="error" class="org.scalastyle.scalariform.ObjectNamesChecker" enabled="true"> + <parameters> + <parameter name="regex"><![CDATA[[A-Z][A-Za-z]*]]></parameter> + </parameters> + </check> + <check level="error" class="org.scalastyle.scalariform.PackageObjectNamesChecker" enabled="true"> + <parameters> + <parameter name="regex"><![CDATA[^[a-z][A-Za-z]*$]]></parameter> + </parameters> + </check> + <check level="error" class="org.scalastyle.scalariform.EqualsHashCodeChecker" enabled="false"></check> + <!-- <check level="error" class="org.scalastyle.scalariform.IllegalImportsChecker" enabled="true"> --> + <!-- <parameters> --> + <!-- <parameter name="illegalImports"><![CDATA[sun._,java.awt._]]></parameter> --> + <!-- </parameters> --> + <!-- </check> --> + <check level="error" class="org.scalastyle.scalariform.ParameterNumberChecker" enabled="true"> + <parameters> + <parameter name="maxParameters"><![CDATA[10]]></parameter> + </parameters> + </check> + <!-- <check level="error" class="org.scalastyle.scalariform.MagicNumberChecker" enabled="true"> --> + <!-- <parameters> --> + <!-- <parameter name="ignore"><![CDATA[-1,0,1,2,3]]></parameter> --> + <!-- </parameters> --> + <!-- </check> --> + <check level="error" class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" enabled="false"></check> + <check level="error" class="org.scalastyle.scalariform.NoWhitespaceAfterLeftBracketChecker" enabled="false"></check> + <!-- <check level="error" class="org.scalastyle.scalariform.ReturnChecker" enabled="true"></check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.NullChecker" enabled="true"></check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.NoCloneChecker" enabled="true"></check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.NoFinalizeChecker" enabled="true"></check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.CovariantEqualsChecker" enabled="true"></check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.StructuralTypeChecker" enabled="true"></check> --> + <!-- <check level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> --> + <!-- <parameters> --> + <!-- <parameter name="regex"><![CDATA[println]]></parameter> --> + <!-- </parameters> --> + <!-- </check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.NumberOfTypesChecker" enabled="true"> --> + <!-- <parameters> --> + <!-- <parameter name="maxTypes"><![CDATA[30]]></parameter> --> + <!-- </parameters> --> + <!-- </check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.CyclomaticComplexityChecker" enabled="true"> --> + <!-- <parameters> --> + <!-- <parameter name="maximum"><![CDATA[10]]></parameter> --> + <!-- </parameters> --> + <!-- </check> --> + <check level="error" class="org.scalastyle.scalariform.UppercaseLChecker" enabled="true"></check> + <check level="error" class="org.scalastyle.scalariform.SimplifyBooleanExpressionChecker" enabled="false"></check> + <check level="error" class="org.scalastyle.scalariform.IfBraceChecker" enabled="true"> + <parameters> + <parameter name="singleLineAllowed"><![CDATA[true]]></parameter> + <parameter name="doubleLineAllowed"><![CDATA[true]]></parameter> + </parameters> + </check> + <!-- <check level="error" class="org.scalastyle.scalariform.MethodLengthChecker" enabled="true"> --> + <!-- <parameters> --> + <!-- <parameter name="maxLength"><![CDATA[50]]></parameter> --> + <!-- </parameters> --> + <!-- </check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.MethodNamesChecker" enabled="true"> --> + <!-- <parameters> --> + <!-- <parameter name="regex"><![CDATA[^[a-z][A-Za-z0-9]*$]]></parameter> --> + <!-- </parameters> --> + <!-- </check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.NumberOfMethodsInTypeChecker" enabled="true"> --> + <!-- <parameters> --> + <!-- <parameter name="maxMethods"><![CDATA[30]]></parameter> --> + <!-- </parameters> --> + <!-- </check> --> + <!-- <check level="error" class="org.scalastyle.scalariform.PublicMethodsHaveTypeChecker" enabled="true"></check> --> + <check level="error" class="org.scalastyle.file.NewLineAtEofChecker" enabled="true"></check> + <check level="error" class="org.scalastyle.file.NoNewLineAtEofChecker" enabled="false"></check> +</scalastyle> diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala index 54813934b8c9570a1c3acc6b27532753063ea648..6a45bc2f8acbca587039e7ef8cac414cfa8f69cd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala @@ -47,7 +47,8 @@ object MasterFailureTest extends Logging { def main(args: Array[String]) { if (args.size < 2) { println( - "Usage: MasterFailureTest <local/HDFS directory> <# batches> [<batch size in milliseconds>]") + "Usage: MasterFailureTest <local/HDFS directory> <# batches> " + + "[<batch size in milliseconds>]") System.exit(1) } val directory = args(0) @@ -186,7 +187,8 @@ object MasterFailureTest extends Logging { // Setup the streaming computation with the given operation System.clearProperty("spark.driver.port") - val ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, Map()) + val ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, + Map()) ssc.checkpoint(checkpointDir.toString) val inputStream = ssc.textFileStream(testDir.toString) val operatedStream = operation(inputStream) @@ -287,7 +289,7 @@ object MasterFailureTest extends Logging { private def verifyOutput[T: ClassTag](output: Seq[T], expectedOutput: Seq[T]) { // Verify whether expected outputs do not consecutive batches with same output for (i <- 0 until expectedOutput.size - 1) { - assert(expectedOutput(i) != expectedOutput(i+1), + assert(expectedOutput(i) != expectedOutput(i + 1), "Expected output has consecutive duplicate sequence of values") } @@ -384,9 +386,9 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) Thread.sleep(5000) // To make sure that all the streaming context has been set up for (i <- 0 until input.size) { // Write the data to a local file and then move it to the target test directory - val localFile = new File(localTestDir, (i+1).toString) - val hadoopFile = new Path(testDir, (i+1).toString) - val tempHadoopFile = new Path(testDir, ".tmp_" + (i+1).toString) + val localFile = new File(localTestDir, (i + 1).toString) + val hadoopFile = new Path(testDir, (i + 1).toString) + val tempHadoopFile = new Path(testDir, ".tmp_" + (i + 1).toString) FileUtils.writeStringToFile(localFile, input(i).toString + "\n") var tries = 0 var done = false @@ -400,7 +402,8 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) } catch { case ioe: IOException => { fs = testDir.getFileSystem(new Configuration()) - logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe) + logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", + ioe) } } } 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 4886cd6ea8a64a1269dfd1f6fdb229d074b6f583..420522433e1e8af469960330ec3f823bc8361307 100644 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -152,7 +152,8 @@ object JavaAPICompletenessChecker { if (parameters(0).name == classOf[Tuple2[_, _]].getName) { val tupleParams = parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs) - ParameterizedType("org.apache.spark.streaming.api.java.JavaPairDStream", tupleParams) + ParameterizedType("org.apache.spark.streaming.api.java.JavaPairDStream", + tupleParams) } else { ParameterizedType("org.apache.spark.streaming.api.java.JavaDStream", parameters.map(applySubs)) @@ -175,7 +176,8 @@ object JavaAPICompletenessChecker { ParameterizedType("org.apache.spark.api.java.function.VoidFunction", parameters.dropRight(1).map(applySubs)) } else { - ParameterizedType("org.apache.spark.api.java.function.Function", parameters.map(applySubs)) + ParameterizedType("org.apache.spark.api.java.function.Function", + parameters.map(applySubs)) } case _ => ParameterizedType(renameSubstitutions.getOrElse(name, name),