Merge pull request #557 from ScrapCodes/style. Closes #557.

SPARK-1058, Fix Style Errors and Add Scala Style to Spark Build.

Author: Patrick Wendell <pwendell@gmail.com>
Author: Prashant Sharma <scrapcodes@gmail.com>

== Merge branch commits ==

commit 1a8bd1c059b842cb95cc246aaea74a79fec684f4
Author: Prashant Sharma <scrapcodes@gmail.com>
Date:   Sun Feb 9 17:39:07 2014 +0530

    scala style fixes

commit f91709887a8e0b608c5c2b282db19b8a44d53a43
Author: Patrick Wendell <pwendell@gmail.com>
Date:   Fri Jan 24 11:22:53 2014 -0800

    Adding scalastyle snapshot
This commit is contained in:
Patrick Wendell 2014-02-09 10:09:19 -08:00
Родитель b6dba10ae5
Коммит b69f8b2a01
119 изменённых файлов: 795 добавлений и 460 удалений

Просмотреть файл

@ -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)

Просмотреть файл

@ -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 {

Просмотреть файл

@ -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)

Просмотреть файл

@ -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)

Просмотреть файл

@ -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 {

Просмотреть файл

@ -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

Просмотреть файл

@ -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()

Просмотреть файл

@ -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

Просмотреть файл

@ -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)

Просмотреть файл

@ -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)

Просмотреть файл

@ -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

Просмотреть файл

@ -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")
}

Просмотреть файл

@ -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

Просмотреть файл

@ -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)
}

Просмотреть файл

@ -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)

Просмотреть файл

@ -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]()

Просмотреть файл

@ -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) =>

Просмотреть файл

@ -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
}

Просмотреть файл

@ -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
}
}
}
}

Просмотреть файл

@ -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}

Просмотреть файл

@ -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")

Просмотреть файл

@ -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>

Просмотреть файл

@ -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)
}

Просмотреть файл

@ -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 = {

Просмотреть файл

@ -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 {
""
}
)
}
}

Просмотреть файл

@ -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()
})

Просмотреть файл

@ -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
}

Просмотреть файл

@ -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.
*

Просмотреть файл

@ -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()
}

Просмотреть файл

@ -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

Просмотреть файл

@ -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()

Просмотреть файл

@ -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)
})
}

Просмотреть файл

@ -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")
}

Просмотреть файл

@ -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

Просмотреть файл

@ -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))

Просмотреть файл

@ -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()

Просмотреть файл

@ -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
}

Просмотреть файл

@ -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(

Просмотреть файл

@ -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 + ")")
}

Просмотреть файл

@ -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) {

Просмотреть файл

@ -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 _ =>
}

Просмотреть файл

@ -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) {

Просмотреть файл

@ -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 {

Просмотреть файл

@ -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

Просмотреть файл

@ -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()

Просмотреть файл

@ -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)
}
}

Просмотреть файл

@ -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)
}

Просмотреть файл

@ -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) {

Просмотреть файл

@ -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. */

Просмотреть файл

@ -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 = {

Просмотреть файл

@ -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)

Просмотреть файл

@ -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.

Просмотреть файл

@ -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]

Просмотреть файл

@ -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"))
}

Просмотреть файл

@ -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)))
}

Просмотреть файл

@ -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))
}

Просмотреть файл

@ -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 {

Просмотреть файл

@ -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

Просмотреть файл

@ -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")

Просмотреть файл

@ -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")

Просмотреть файл

@ -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
}
}

Просмотреть файл

@ -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
}

Просмотреть файл

@ -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
}

Просмотреть файл

@ -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

Просмотреть файл

@ -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>

Просмотреть файл

@ -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)
}

Просмотреть файл

@ -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",

Просмотреть файл

@ -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

Просмотреть файл

@ -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>

Просмотреть файл

@ -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("")

Просмотреть файл

@ -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)

Просмотреть файл

@ -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)
}
}
}
}

Просмотреть файл

@ -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

Просмотреть файл

@ -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]) {

Просмотреть файл

@ -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

Просмотреть файл

@ -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 {

Просмотреть файл

@ -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()")
}
}
/**

Просмотреть файл

@ -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.
*/
@ -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. */

Просмотреть файл

@ -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()

Просмотреть файл

@ -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

Просмотреть файл

@ -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
}
}

Просмотреть файл

@ -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

Просмотреть файл

@ -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
}
}

Просмотреть файл

@ -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

Просмотреть файл

@ -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))
}
})

Просмотреть файл

@ -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,

Просмотреть файл

@ -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) {

Просмотреть файл

@ -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

Просмотреть файл

@ -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)
}
/**

Просмотреть файл

@ -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 = {

Просмотреть файл

@ -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,

Просмотреть файл

@ -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])

Просмотреть файл

@ -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

Просмотреть файл

@ -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 = {

Просмотреть файл

@ -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] = {

Просмотреть файл

@ -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)

Просмотреть файл

@ -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

Просмотреть файл

@ -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 => {

Просмотреть файл

@ -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,8 +145,8 @@ 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)

Просмотреть файл

@ -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)

Некоторые файлы не были показаны из-за слишком большого количества измененных файлов Показать больше