Merge branch 'master' into blockmanager_ui

Conflicts:
	core/src/main/scala/spark/RDD.scala
This commit is contained in:
Imran Rashid 2013-01-22 12:02:27 -08:00
commit 905c720e5e
159 changed files with 11631 additions and 470 deletions

2
.gitignore vendored
View file

@ -12,6 +12,7 @@ third_party/libmesos.so
third_party/libmesos.dylib
conf/java-opts
conf/spark-env.sh
conf/streaming-env.sh
conf/log4j.properties
docs/_site
docs/api
@ -31,4 +32,5 @@ project/plugins/src_managed/
logs/
log/
spark-tests.log
streaming-tests.log
dependency-reduced-pom.xml

View file

@ -1,8 +1,8 @@
# Set everything to be logged to the console
# Set everything to be logged to the file bagel/target/unit-tests.log
log4j.rootCategory=INFO, file
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
log4j.appender.file.file=spark-tests.log
log4j.appender.file.file=bagel/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n

View file

@ -14,6 +14,7 @@ import akka.util.duration._
import spark.storage.BlockManager
import spark.storage.StorageLevel
import util.{TimeStampedHashSet, MetadataCleaner, TimeStampedHashMap}
private[spark] sealed trait CacheTrackerMessage
@ -30,7 +31,7 @@ private[spark] case object StopCacheTracker extends CacheTrackerMessage
private[spark] class CacheTrackerActor extends Actor with Logging {
// TODO: Should probably store (String, CacheType) tuples
private val locs = new HashMap[Int, Array[List[String]]]
private val locs = new TimeStampedHashMap[Int, Array[List[String]]]
/**
* A map from the slave's host name to its cache size.
@ -38,6 +39,8 @@ private[spark] class CacheTrackerActor extends Actor with Logging {
private val slaveCapacity = new HashMap[String, Long]
private val slaveUsage = new HashMap[String, Long]
private val metadataCleaner = new MetadataCleaner("CacheTrackerActor", locs.clearOldValues)
private def getCacheUsage(host: String): Long = slaveUsage.getOrElse(host, 0L)
private def getCacheCapacity(host: String): Long = slaveCapacity.getOrElse(host, 0L)
private def getCacheAvailable(host: String): Long = getCacheCapacity(host) - getCacheUsage(host)
@ -86,6 +89,7 @@ private[spark] class CacheTrackerActor extends Actor with Logging {
case StopCacheTracker =>
logInfo("Stopping CacheTrackerActor")
sender ! true
metadataCleaner.cancel()
context.stop(self)
}
}
@ -109,11 +113,15 @@ private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, b
actorSystem.actorFor(url)
}
val registeredRddIds = new HashSet[Int]
// TODO: Consider removing this HashSet completely as locs CacheTrackerActor already
// keeps track of registered RDDs
val registeredRddIds = new TimeStampedHashSet[Int]
// Remembers which splits are currently being loaded (on worker nodes)
val loading = new HashSet[String]
val metadataCleaner = new MetadataCleaner("CacheTracker", registeredRddIds.clearOldValues)
// Send a message to the trackerActor and get its result within a default timeout, or
// throw a SparkException if this fails.
def askTracker(message: Any): Any = {

View file

@ -17,6 +17,7 @@ import akka.util.duration._
import spark.scheduler.MapStatus
import spark.storage.BlockManagerId
import spark.util.{MetadataCleaner, TimeStampedHashMap}
private[spark] sealed trait MapOutputTrackerMessage
@ -44,7 +45,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
val timeout = 10.seconds
var mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]
var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
// Incremented every time a fetch fails so that client nodes know to clear
// their cache of map output locations if this happens.
@ -53,7 +54,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
// Cache a serialized version of the output statuses for each shuffle to send them out faster
var cacheGeneration = generation
val cachedSerializedStatuses = new HashMap[Int, Array[Byte]]
val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]
var trackerActor: ActorRef = if (isMaster) {
val actor = actorSystem.actorOf(Props(new MapOutputTrackerActor(this)), name = actorName)
@ -64,6 +65,8 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
actorSystem.actorFor(url)
}
val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup)
// Send a message to the trackerActor and get its result within a default timeout, or
// throw a SparkException if this fails.
def askTracker(message: Any): Any = {
@ -84,14 +87,14 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
}
def registerShuffle(shuffleId: Int, numMaps: Int) {
if (mapStatuses.get(shuffleId) != null) {
if (mapStatuses.get(shuffleId) != None) {
throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice")
}
mapStatuses.put(shuffleId, new Array[MapStatus](numMaps))
}
def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) {
var array = mapStatuses.get(shuffleId)
var array = mapStatuses(shuffleId)
array.synchronized {
array(mapId) = status
}
@ -108,7 +111,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
}
def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) {
var array = mapStatuses.get(shuffleId)
var array = mapStatuses(shuffleId)
if (array != null) {
array.synchronized {
if (array(mapId) != null && array(mapId).address == bmAddress) {
@ -126,7 +129,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
// Called on possibly remote nodes to get the server URIs and output sizes for a given shuffle
def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = {
val statuses = mapStatuses.get(shuffleId)
val statuses = mapStatuses.get(shuffleId).orNull
if (statuses == null) {
logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them")
fetching.synchronized {
@ -139,8 +142,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
case e: InterruptedException =>
}
}
return MapOutputTracker.convertMapStatuses(shuffleId, reduceId,
mapStatuses.get(shuffleId))
return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, mapStatuses(shuffleId))
} else {
fetching += shuffleId
}
@ -168,9 +170,15 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
}
}
def cleanup(cleanupTime: Long) {
mapStatuses.clearOldValues(cleanupTime)
cachedSerializedStatuses.clearOldValues(cleanupTime)
}
def stop() {
communicate(StopMapOutputTracker)
mapStatuses.clear()
metadataCleaner.cancel()
trackerActor = null
}
@ -196,7 +204,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
generationLock.synchronized {
if (newGen > generation) {
logInfo("Updating generation to " + newGen + " and clearing cache")
mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]
mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
generation = newGen
}
}
@ -214,7 +222,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
case Some(bytes) =>
return bytes
case None =>
statuses = mapStatuses.get(shuffleId)
statuses = mapStatuses(shuffleId)
generationGotten = generation
}
}

View file

@ -199,9 +199,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
/**
* Merge the values for each key using an associative reduce function. This will also perform
* the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce.
* Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each
* pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
* (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD.
*/
def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = {
this.cogroup(other, partitioner).flatMapValues {
@ -485,18 +485,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
}
/**
* Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat`
* (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.
*/
def saveAsNewAPIHadoopFile(
path: String,
keyClass: Class[_],
valueClass: Class[_],
outputFormatClass: Class[_ <: NewOutputFormat[_, _]]) {
saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass, new Configuration)
}
/**
* Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat`
* (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.
@ -506,7 +494,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
keyClass: Class[_],
valueClass: Class[_],
outputFormatClass: Class[_ <: NewOutputFormat[_, _]],
conf: Configuration) {
conf: Configuration = self.context.hadoopConfiguration) {
val job = new NewAPIHadoopJob(conf)
job.setOutputKeyClass(keyClass)
job.setOutputValueClass(valueClass)
@ -557,7 +545,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
keyClass: Class[_],
valueClass: Class[_],
outputFormatClass: Class[_ <: OutputFormat[_, _]],
conf: JobConf = new JobConf) {
conf: JobConf = new JobConf(self.context.hadoopConfiguration)) {
conf.setOutputKeyClass(keyClass)
conf.setOutputValueClass(valueClass)
// conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug
@ -661,24 +649,23 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
}
private[spark]
class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev.context) {
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
override val partitioner = prev.partitioner
override def compute(split: Split, taskContext: TaskContext) =
prev.iterator(split, taskContext).map{case (k, v) => (k, f(v))}
class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U)
extends RDD[(K, U)](prev) {
override def getSplits = firstParent[(K, V)].splits
override val partitioner = firstParent[(K, V)].partitioner
override def compute(split: Split, context: TaskContext) =
firstParent[(K, V)].iterator(split, context).map{ case (k, v) => (k, f(v)) }
}
private[spark]
class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U])
extends RDD[(K, U)](prev.context) {
extends RDD[(K, U)](prev) {
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
override val partitioner = prev.partitioner
override def compute(split: Split, taskContext: TaskContext) = {
prev.iterator(split, taskContext).flatMap { case (k, v) => f(v).map(x => (k, x)) }
override def getSplits = firstParent[(K, V)].splits
override val partitioner = firstParent[(K, V)].partitioner
override def compute(split: Split, context: TaskContext) = {
firstParent[(K, V)].iterator(split, context).flatMap { case (k, v) => f(v).map(x => (k, x)) }
}
}

View file

@ -2,6 +2,7 @@ package spark
import scala.collection.immutable.NumericRange
import scala.collection.mutable.ArrayBuffer
import scala.collection.Map
private[spark] class ParallelCollectionSplit[T: ClassManifest](
val rddId: Long,
@ -22,30 +23,40 @@ private[spark] class ParallelCollectionSplit[T: ClassManifest](
}
private[spark] class ParallelCollection[T: ClassManifest](
sc: SparkContext,
@transient sc : SparkContext,
@transient data: Seq[T],
numSlices: Int)
extends RDD[T](sc) {
numSlices: Int,
locationPrefs : Map[Int,Seq[String]])
extends RDD[T](sc, Nil) {
// TODO: Right now, each split sends along its full data, even if later down the RDD chain it gets
// cached. It might be worthwhile to write the data to a file in the DFS and read it in the split
// instead.
// UPDATE: A parallel collection can be checkpointed to HDFS, which achieves this goal.
@transient
val splits_ = {
var splits_ : Array[Split] = {
val slices = ParallelCollection.slice(data, numSlices).toArray
slices.indices.map(i => new ParallelCollectionSplit(id, i, slices(i))).toArray
}
override def splits = splits_.asInstanceOf[Array[Split]]
override def getSplits = splits_.asInstanceOf[Array[Split]]
override def compute(s: Split, taskContext: TaskContext) =
override def compute(s: Split, context: TaskContext) =
s.asInstanceOf[ParallelCollectionSplit[T]].iterator
override def preferredLocations(s: Split): Seq[String] = Nil
override def getPreferredLocations(s: Split): Seq[String] = {
locationPrefs.get(s.index) match {
case Some(s) => s
case _ => Nil
}
}
override val dependencies: List[Dependency[_]] = Nil
override def clearDependencies() {
splits_ = null
}
}
private object ParallelCollection {
/**
* Slice a collection into numSlices sub-collections. One extra thing we do here is to treat Range

View file

@ -1,10 +1,8 @@
package spark
import java.io.EOFException
import java.io.ObjectInputStream
import java.io.{ObjectOutputStream, IOException, EOFException, ObjectInputStream}
import java.net.URL
import java.util.Random
import java.util.Date
import java.util.{Date, Random}
import java.util.{HashMap => JHashMap}
import java.util.concurrent.atomic.AtomicLong
@ -13,6 +11,7 @@ import scala.collection.JavaConversions.mapAsScalaMap
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.BytesWritable
import org.apache.hadoop.io.NullWritable
import org.apache.hadoop.io.Text
@ -73,44 +72,45 @@ import SparkContext._
* [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details
* on RDD internals.
*/
abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serializable {
abstract class RDD[T: ClassManifest](
@transient var sc: SparkContext,
var dependencies_ : List[Dependency[_]]
) extends Serializable with Logging {
// Methods that must be implemented by subclasses:
/** Set of partitions in this RDD. */
def splits: Array[Split]
def this(@transient oneParent: RDD[_]) =
this(oneParent.context , List(new OneToOneDependency(oneParent)))
// =======================================================================
// Methods that should be implemented by subclasses of RDD
// =======================================================================
/** Function for computing a given partition. */
def compute(split: Split, context: TaskContext): Iterator[T]
/** How this RDD depends on any parent RDDs. */
@transient val dependencies: List[Dependency[_]]
/** Set of partitions in this RDD. */
protected def getSplits(): Array[Split]
// Methods available on all RDDs:
/** How this RDD depends on any parent RDDs. */
protected def getDependencies(): List[Dependency[_]] = dependencies_
// A friendly name for this RDD
var name: String = null
/** Record user function generating this RDD. */
private[spark] val origin = Utils.getSparkCallSite
/** Optionally overridden by subclasses to specify placement preferences. */
protected def getPreferredLocations(split: Split): Seq[String] = Nil
/** Optionally overridden by subclasses to specify how they are partitioned. */
val partitioner: Option[Partitioner] = None
/** Optionally overridden by subclasses to specify placement preferences. */
def preferredLocations(split: Split): Seq[String] = Nil
/** The [[spark.SparkContext]] that this RDD was created on. */
def context = sc
private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T]
// =======================================================================
// Methods and fields available on all RDDs
// =======================================================================
/** A unique ID for this RDD (within its SparkContext). */
val id = sc.newRddId()
// Variables relating to persistence
private var storageLevel: StorageLevel = StorageLevel.NONE
/* Assign a name to this RDD */
def setName(_name: String) = {
name = _name
@ -142,22 +142,39 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
/** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
def getStorageLevel = storageLevel
private[spark] def checkpoint(level: StorageLevel = StorageLevel.MEMORY_AND_DISK_2): RDD[T] = {
if (!level.useDisk && level.replication < 2) {
throw new Exception("Cannot checkpoint without using disk or replication (level requested was " + level + ")")
/**
* Get the preferred location of a split, taking into account whether the
* RDD is checkpointed or not.
*/
final def preferredLocations(split: Split): Seq[String] = {
if (isCheckpointed) {
checkpointData.get.getPreferredLocations(split)
} else {
getPreferredLocations(split)
}
}
// This is a hack. Ideally this should re-use the code used by the CacheTracker
// to generate the key.
def getSplitKey(split: Split) = "rdd_%d_%d".format(this.id, split.index)
/**
* Get the array of splits of this RDD, taking into account whether the
* RDD is checkpointed or not.
*/
final def splits: Array[Split] = {
if (isCheckpointed) {
checkpointData.get.getSplits
} else {
getSplits
}
}
persist(level)
sc.runJob(this, (iter: Iterator[T]) => {} )
val p = this.partitioner
new BlockRDD[T](sc, splits.map(getSplitKey).toArray) {
override val partitioner = p
/**
* Get the list of dependencies of this RDD, taking into account whether the
* RDD is checkpointed or not.
*/
final def dependencies: List[Dependency[_]] = {
if (isCheckpointed) {
dependencies_
} else {
getDependencies
}
}
@ -167,7 +184,9 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
* subclasses of RDD.
*/
final def iterator(split: Split, context: TaskContext): Iterator[T] = {
if (storageLevel != StorageLevel.NONE) {
if (isCheckpointed) {
checkpointData.get.iterator(split, context)
} else if (storageLevel != StorageLevel.NONE) {
SparkEnv.get.cacheTracker.getOrCompute[T](this, split, context, storageLevel)
} else {
compute(split, context)
@ -539,4 +558,84 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
private[spark] def collectPartitions(): Array[Array[T]] = {
sc.runJob(this, (iter: Iterator[T]) => iter.toArray)
}
/**
* Mark this RDD for checkpointing. It will be saved to a file inside the checkpoint
* directory set with SparkContext.setCheckpointDir() and all references to its parent
* RDDs will be removed. This function must be called before any job has been
* executed on this RDD. It is strongly recommended that this RDD is persisted in
* memory, otherwise saving it on a file will require recomputation.
*/
def checkpoint() {
if (context.checkpointDir.isEmpty) {
throw new Exception("Checkpoint directory has not been set in the SparkContext")
} else if (checkpointData.isEmpty) {
checkpointData = Some(new RDDCheckpointData(this))
checkpointData.get.markForCheckpoint()
}
}
/**
* Return whether this RDD has been checkpointed or not
*/
def isCheckpointed(): Boolean = {
if (checkpointData.isDefined) checkpointData.get.isCheckpointed() else false
}
/**
* Gets the name of the file to which this RDD was checkpointed
*/
def getCheckpointFile(): Option[String] = {
if (checkpointData.isDefined) checkpointData.get.getCheckpointFile() else None
}
// =======================================================================
// Other internal methods and fields
// =======================================================================
private var storageLevel: StorageLevel = StorageLevel.NONE
/** Record user function generating this RDD. */
private[spark] val origin = Utils.getSparkCallSite
private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T]
private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None
/** Returns the first parent RDD */
protected[spark] def firstParent[U: ClassManifest] = {
dependencies.head.rdd.asInstanceOf[RDD[U]]
}
/** The [[spark.SparkContext]] that this RDD was created on. */
def context = sc
/**
* Performs the checkpointing of this RDD by saving this . It is called by the DAGScheduler
* after a job using this RDD has completed (therefore the RDD has been materialized and
* potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs.
*/
protected[spark] def doCheckpoint() {
if (checkpointData.isDefined) checkpointData.get.doCheckpoint()
dependencies.foreach(_.rdd.doCheckpoint())
}
/**
* Changes the dependencies of this RDD from its original parents to the new RDD
* (`newRDD`) created from the checkpoint file.
*/
protected[spark] def changeDependencies(newRDD: RDD[_]) {
clearDependencies()
dependencies_ = List(new OneToOneDependency(newRDD))
}
/**
* Clears the dependencies of this RDD. This method must ensure that all references
* to the original parent RDDs is removed to enable the parent RDDs to be garbage
* collected. Subclasses of RDD may override this method for implementing their own cleaning
* logic. See [[spark.rdd.UnionRDD]] and [[spark.rdd.ShuffledRDD]] to get a better idea.
*/
protected[spark] def clearDependencies() {
dependencies_ = null
}
}

View file

@ -0,0 +1,105 @@
package spark
import org.apache.hadoop.fs.Path
import rdd.{CheckpointRDD, CoalescedRDD}
import scheduler.{ResultTask, ShuffleMapTask}
/**
* Enumeration to manage state transitions of an RDD through checkpointing
* [ Initialized --> marked for checkpointing --> checkpointing in progress --> checkpointed ]
*/
private[spark] object CheckpointState extends Enumeration {
type CheckpointState = Value
val Initialized, MarkedForCheckpoint, CheckpointingInProgress, Checkpointed = Value
}
/**
* 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 splits, iterator and preferred locations
* of the checkpointed RDD.
*/
private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
extends Logging with Serializable {
import CheckpointState._
// The checkpoint state of the associated RDD.
var cpState = Initialized
// The file to which the associated RDD has been checkpointed to
@transient var cpFile: Option[String] = None
// The CheckpointRDD created from the checkpoint file, that is, the new parent the associated RDD.
@transient var cpRDD: Option[RDD[T]] = None
// Mark the RDD for checkpointing
def markForCheckpoint() {
RDDCheckpointData.synchronized {
if (cpState == Initialized) cpState = MarkedForCheckpoint
}
}
// Is the RDD already checkpointed
def isCheckpointed(): Boolean = {
RDDCheckpointData.synchronized { cpState == Checkpointed }
}
// Get the file to which this RDD was checkpointed to as an Option
def getCheckpointFile(): Option[String] = {
RDDCheckpointData.synchronized { cpFile }
}
// Do the checkpointing of the RDD. Called after the first job using that RDD is over.
def doCheckpoint() {
// If it is marked for checkpointing AND checkpointing is not already in progress,
// then set it to be in progress, else return
RDDCheckpointData.synchronized {
if (cpState == MarkedForCheckpoint) {
cpState = CheckpointingInProgress
} else {
return
}
}
// Save to file, and reload it as an RDD
val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id).toString
rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path) _)
val newRDD = new CheckpointRDD[T](rdd.context, path)
// Change the dependencies and splits of the RDD
RDDCheckpointData.synchronized {
cpFile = Some(path)
cpRDD = Some(newRDD)
rdd.changeDependencies(newRDD)
cpState = Checkpointed
RDDCheckpointData.clearTaskCaches()
logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id)
}
}
// Get preferred location of a split after checkpointing
def getPreferredLocations(split: Split) = {
RDDCheckpointData.synchronized {
cpRDD.get.preferredLocations(split)
}
}
def getSplits: Array[Split] = {
RDDCheckpointData.synchronized {
cpRDD.get.splits
}
}
// Get iterator. This is called at the worker nodes.
def iterator(split: Split, context: TaskContext): Iterator[T] = {
rdd.firstParent[T].iterator(split, context)
}
}
private[spark] object RDDCheckpointData {
def clearTaskCaches() {
ShuffleMapTask.clearCache()
ResultTask.clearCache()
}
}

View file

@ -4,10 +4,12 @@ import java.io._
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicInteger
import java.net.{URI, URLClassLoader}
import java.lang.ref.WeakReference
import scala.collection.Map
import scala.collection.generic.Growable
import scala.collection.mutable.{ArrayBuffer, HashMap}
import scala.collection.JavaConversions._
import akka.actor.Actor
import akka.actor.Actor._
@ -37,12 +39,8 @@ import spark.broadcast._
import spark.deploy.LocalSparkCluster
import spark.partial.ApproximateEvaluator
import spark.partial.PartialResult
import spark.rdd.HadoopRDD
import spark.rdd.NewHadoopRDD
import spark.rdd.UnionRDD
import spark.scheduler.ShuffleMapTask
import spark.scheduler.DAGScheduler
import spark.scheduler.TaskScheduler
import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD}
import scheduler.{ResultTask, ShuffleMapTask, DAGScheduler, TaskScheduler}
import spark.scheduler.local.LocalScheduler
import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler}
import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
@ -59,10 +57,10 @@ import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend
* @param environment Environment variables to set on worker nodes.
*/
class SparkContext(
master: String,
jobName: String,
val master: String,
val jobName: String,
val sparkHome: String,
jars: Seq[String],
val jars: Seq[String],
environment: Map[String, String])
extends Logging {
@ -195,11 +193,32 @@ class SparkContext(
private var dagScheduler = new DAGScheduler(taskScheduler)
/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
val hadoopConfiguration = {
val conf = new Configuration()
// Explicitly check for S3 environment variables
if (System.getenv("AWS_ACCESS_KEY_ID") != null && System.getenv("AWS_SECRET_ACCESS_KEY") != null) {
conf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
conf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
conf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
conf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
}
// Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar"
for (key <- System.getProperties.toMap[String, String].keys if key.startsWith("spark.hadoop.")) {
conf.set(key.substring("spark.hadoop.".length), System.getProperty(key))
}
val bufferSize = System.getProperty("spark.buffer.size", "65536")
conf.set("io.file.buffer.size", bufferSize)
conf
}
private[spark] var checkpointDir: Option[String] = None
// Methods for creating RDDs
/** Distribute a local Scala collection to form an RDD. */
def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
new ParallelCollection[T](this, seq, numSlices)
new ParallelCollection[T](this, seq, numSlices, Map[Int, Seq[String]]())
}
/** Distribute a local Scala collection to form an RDD. */
@ -207,6 +226,14 @@ class SparkContext(
parallelize(seq, numSlices)
}
/** Distribute a local Scala collection to form an RDD, with one or more
* location preferences (hostnames of Spark nodes) for each object.
* Create a new partition for each collection item. */
def makeRDD[T: ClassManifest](seq: Seq[(T, Seq[String])]): RDD[T] = {
val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
new ParallelCollection[T](this, seq.map(_._1), seq.size, indexToPrefs)
}
/**
* Read a text file from HDFS, a local file system (available on all nodes), or any
* Hadoop-supported file system URI, and return it as an RDD of Strings.
@ -239,10 +266,8 @@ class SparkContext(
valueClass: Class[V],
minSplits: Int = defaultMinSplits
) : RDD[(K, V)] = {
val conf = new JobConf()
val conf = new JobConf(hadoopConfiguration)
FileInputFormat.setInputPaths(conf, path)
val bufferSize = System.getProperty("spark.buffer.size", "65536")
conf.set("io.file.buffer.size", bufferSize)
new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits)
}
@ -283,8 +308,7 @@ class SparkContext(
path,
fm.erasure.asInstanceOf[Class[F]],
km.erasure.asInstanceOf[Class[K]],
vm.erasure.asInstanceOf[Class[V]],
new Configuration)
vm.erasure.asInstanceOf[Class[V]])
}
/**
@ -296,7 +320,7 @@ class SparkContext(
fClass: Class[F],
kClass: Class[K],
vClass: Class[V],
conf: Configuration): RDD[(K, V)] = {
conf: Configuration = hadoopConfiguration): RDD[(K, V)] = {
val job = new NewHadoopJob(conf)
NewFileInputFormat.addInputPath(job, new Path(path))
val updatedConf = job.getConfiguration
@ -308,7 +332,7 @@ class SparkContext(
* and extra configuration options to pass to the input format.
*/
def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]](
conf: Configuration,
conf: Configuration = hadoopConfiguration,
fClass: Class[F],
kClass: Class[K],
vClass: Class[V]): RDD[(K, V)] = {
@ -373,6 +397,13 @@ class SparkContext(
.flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes))
}
protected[spark] def checkpointFile[T: ClassManifest](
path: String
): RDD[T] = {
new CheckpointRDD[T](this, path)
}
/** Build the union of a list of RDDs. */
def union[T: ClassManifest](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds)
@ -479,17 +510,22 @@ class SparkContext(
/** Shut down the SparkContext. */
def stop() {
dagScheduler.stop()
dagScheduler = null
taskScheduler = null
// TODO: Cache.stop()?
env.stop()
// Clean up locally linked files
clearFiles()
clearJars()
SparkEnv.set(null)
ShuffleMapTask.clearCache()
logInfo("Successfully stopped SparkContext")
if (dagScheduler != null) {
dagScheduler.stop()
dagScheduler = null
taskScheduler = null
// TODO: Cache.stop()?
env.stop()
// Clean up locally linked files
clearFiles()
clearJars()
SparkEnv.set(null)
ShuffleMapTask.clearCache()
ResultTask.clearCache()
logInfo("Successfully stopped SparkContext")
} else {
logInfo("SparkContext already stopped")
}
}
/**
@ -526,6 +562,7 @@ class SparkContext(
val start = System.nanoTime
val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal)
logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s")
rdd.doCheckpoint()
result
}
@ -582,6 +619,26 @@ class SparkContext(
return f
}
/**
* Set the directory under which RDDs are going to be checkpointed. The directory must
* be a HDFS path if running on a cluster. If the directory does not exist, it will
* be created. If the directory exists and useExisting is set to true, then the
* exisiting directory will be used. Otherwise an exception will be thrown to
* prevent accidental overriding of checkpoint files in the existing directory.
*/
def setCheckpointDir(dir: String, useExisting: Boolean = false) {
val path = new Path(dir)
val fs = path.getFileSystem(new Configuration())
if (!useExisting) {
if (fs.exists(path)) {
throw new Exception("Checkpoint directory '" + path + "' already exists.")
} else {
fs.mkdirs(path)
}
}
checkpointDir = Some(dir)
}
/** Default level of parallelism to use when not given by user (e.g. for reduce tasks) */
def defaultParallelism: Int = taskScheduler.defaultParallelism
@ -603,6 +660,7 @@ class SparkContext(
* various Spark features.
*/
object SparkContext {
implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] {
def addInPlace(t1: Double, t2: Double): Double = t1 + t2
def zero(initialValue: Double) = 0.0

View file

@ -1,7 +1,7 @@
package spark
import java.io._
import java.net.{NetworkInterface, InetAddress, URL, URI}
import java.net.{NetworkInterface, InetAddress, Inet4Address, URL, URI}
import java.util.{Locale, Random, UUID}
import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor}
import org.apache.hadoop.conf.Configuration
@ -251,7 +251,8 @@ private object Utils extends Logging {
// Address resolves to something like 127.0.1.1, which happens on Debian; try to find
// a better address using the local network interfaces
for (ni <- NetworkInterface.getNetworkInterfaces) {
for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && !addr.isLoopbackAddress) {
for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress &&
!addr.isLoopbackAddress && addr.isInstanceOf[Inet4Address]) {
// We've found an address that looks reasonable!
logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" +
" a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress +
@ -315,7 +316,7 @@ private object Utils extends Logging {
* millisecond.
*/
def getUsedTimeMs(startTimeMs: Long): String = {
return " " + (System.currentTimeMillis - startTimeMs) + " ms "
return " " + (System.currentTimeMillis - startTimeMs) + " ms"
}
/**

View file

@ -9,6 +9,7 @@ import spark.api.java.JavaPairRDD._
import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
import spark.partial.{PartialResult, BoundedDouble}
import spark.storage.StorageLevel
import com.google.common.base.Optional
trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
@ -306,4 +307,28 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
implicit val kcm: ClassManifest[K] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
JavaPairRDD.fromRDD(rdd.keyBy(f))
}
/**
* Mark this RDD for checkpointing. It will be saved to a file inside the checkpoint
* directory set with SparkContext.setCheckpointDir() and all references to its parent
* RDDs will be removed. This function must be called before any job has been
* executed on this RDD. It is strongly recommended that this RDD is persisted in
* memory, otherwise saving it on a file will require recomputation.
*/
def checkpoint() = rdd.checkpoint()
/**
* Return whether this RDD has been checkpointed or not
*/
def isCheckpointed(): Boolean = rdd.isCheckpointed()
/**
* Gets the name of the file to which this RDD was checkpointed
*/
def getCheckpointFile(): Optional[String] = {
rdd.getCheckpointFile match {
case Some(file) => Optional.of(file)
case _ => Optional.absent()
}
}
}

View file

@ -355,6 +355,40 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
def clearFiles() {
sc.clearFiles()
}
/**
* Returns the Hadoop configuration used for the Hadoop code (e.g. file systems) we reuse.
*/
def hadoopConfiguration(): Configuration = {
sc.hadoopConfiguration
}
/**
* Set the directory under which RDDs are going to be checkpointed. The directory must
* be a HDFS path if running on a cluster. If the directory does not exist, it will
* be created. If the directory exists and useExisting is set to true, then the
* exisiting directory will be used. Otherwise an exception will be thrown to
* prevent accidental overriding of checkpoint files in the existing directory.
*/
def setCheckpointDir(dir: String, useExisting: Boolean) {
sc.setCheckpointDir(dir, useExisting)
}
/**
* Set the directory under which RDDs are going to be checkpointed. The directory must
* be a HDFS path if running on a cluster. If the directory does not exist, it will
* be created. If the directory exists, an exception will be thrown to prevent accidental
* overriding of checkpoint files.
*/
def setCheckpointDir(dir: String) {
sc.setCheckpointDir(dir)
}
protected def checkpointFile[T](path: String): JavaRDD[T] = {
implicit val cm: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
new JavaRDD(sc.checkpointFile(path))
}
}
object JavaSparkContext {

View file

@ -6,8 +6,17 @@ import java.util.Arrays
/**
* A [[spark.Partitioner]] that performs handling of byte arrays, 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
* lifetime of the job (i.e. that it is not re-used as the id of a different partitioning
* function). This can be ensured by using the Python id() function and maintaining a reference
* to the Python partitioning function so that its id() is not reused.
*/
private[spark] class PythonPartitioner(override val numPartitions: Int) extends Partitioner {
private[spark] class PythonPartitioner(
override val numPartitions: Int,
val pyPartitionFunctionId: Long)
extends Partitioner {
override def getPartition(key: Any): Int = {
if (key == null) {
@ -32,7 +41,7 @@ private[spark] class PythonPartitioner(override val numPartitions: Int) extends
override def equals(other: Any): Boolean = other match {
case h: PythonPartitioner =>
h.numPartitions == numPartitions
h.numPartitions == numPartitions && h.pyPartitionFunctionId == pyPartitionFunctionId
case _ =>
false
}

View file

@ -1,7 +1,8 @@
package spark.api.python
import java.io._
import java.util.{List => JList}
import java.net._
import java.util.{List => JList, ArrayList => JArrayList, Collections}
import scala.collection.JavaConversions._
import scala.io.Source
@ -10,29 +11,28 @@ import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
import spark.broadcast.Broadcast
import spark._
import spark.rdd.PipedRDD
import java.util
private[spark] class PythonRDD[T: ClassManifest](
parent: RDD[T],
command: Seq[String],
envVars: java.util.Map[String, String],
preservePartitoning: Boolean,
pythonExec: String,
broadcastVars: java.util.List[Broadcast[Array[Byte]]])
extends RDD[Array[Byte]](parent.context) {
parent: RDD[T],
command: Seq[String],
envVars: java.util.Map[String, String],
preservePartitoning: Boolean,
pythonExec: String,
broadcastVars: JList[Broadcast[Array[Byte]]],
accumulator: Accumulator[JList[Array[Byte]]])
extends RDD[Array[Byte]](parent) {
// Similar to Runtime.exec(), if we are given a single string, split it into words
// using a standard StringTokenizer (i.e. by spaces)
def this(parent: RDD[T], command: String, envVars: java.util.Map[String, String],
preservePartitoning: Boolean, pythonExec: String,
broadcastVars: java.util.List[Broadcast[Array[Byte]]]) =
preservePartitoning: Boolean, pythonExec: String,
broadcastVars: JList[Broadcast[Array[Byte]]],
accumulator: Accumulator[JList[Array[Byte]]]) =
this(parent, PipedRDD.tokenize(command), envVars, preservePartitoning, pythonExec,
broadcastVars)
broadcastVars, accumulator)
override def splits = parent.splits
override val dependencies = List(new OneToOneDependency(parent))
override def getSplits = parent.splits
override val partitioner = if (preservePartitoning) parent.partitioner else None
@ -93,18 +93,30 @@ private[spark] class PythonRDD[T: ClassManifest](
// Return an iterator that read lines from the process's stdout
val stream = new DataInputStream(proc.getInputStream)
return new Iterator[Array[Byte]] {
def next() = {
def next(): Array[Byte] = {
val obj = _nextObj
_nextObj = read()
obj
}
private def read() = {
private def read(): Array[Byte] = {
try {
val length = stream.readInt()
val obj = new Array[Byte](length)
stream.readFully(obj)
obj
if (length != -1) {
val obj = new Array[Byte](length)
stream.readFully(obj)
obj
} else {
// We've finished the data section of the output, but we can still read some
// accumulator updates; let's do that, breaking when we get EOFException
while (true) {
val len2 = stream.readInt()
val update = new Array[Byte](len2)
stream.readFully(update)
accumulator += Collections.singletonList(update)
}
new Array[Byte](0)
}
} catch {
case eof: EOFException => {
val exitStatus = proc.waitFor()
@ -131,9 +143,8 @@ private[spark] class PythonRDD[T: ClassManifest](
* This is used by PySpark's shuffle operations.
*/
private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
RDD[(Array[Byte], Array[Byte])](prev.context) {
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
RDD[(Array[Byte], Array[Byte])](prev) {
override def getSplits = prev.splits
override def compute(split: Split, context: TaskContext) =
prev.iterator(split, context).grouped(2).map {
case Seq(a, b) => (a, b)
@ -238,11 +249,43 @@ private object Pickle {
val APPENDS: Byte = 'e'
}
private class ExtractValue extends spark.api.java.function.Function[(Array[Byte],
Array[Byte]), Array[Byte]] {
override def call(pair: (Array[Byte], Array[Byte])) : Array[Byte] = pair._2
}
private class BytesToString extends spark.api.java.function.Function[Array[Byte], String] {
override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8")
}
/**
* Internal class that acts as an `AccumulatorParam` for Python accumulators. Inside, it
* collects a list of pickled strings that we pass to Python through a socket.
*/
class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int)
extends AccumulatorParam[JList[Array[Byte]]] {
override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList
override def addInPlace(val1: JList[Array[Byte]], val2: JList[Array[Byte]])
: JList[Array[Byte]] = {
if (serverHost == null) {
// This happens on the worker node, where we just want to remember all the updates
val1.addAll(val2)
val1
} else {
// This happens on the master, where we pass the updates to Python through a socket
val socket = new Socket(serverHost, serverPort)
val in = socket.getInputStream
val out = new DataOutputStream(socket.getOutputStream)
out.writeInt(val2.size)
for (array <- val2) {
out.writeInt(array.length)
out.write(array)
}
out.flush()
// Wait for a byte from the Python side as an acknowledgement
val byteRead = in.read()
if (byteRead == -1) {
throw new SparkException("EOF reached before Python server acknowledged")
}
socket.close()
null
}
}
}

View file

@ -11,6 +11,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
import spark._
import spark.storage.StorageLevel
import util.{MetadataCleaner, TimeStampedHashSet}
private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
extends Broadcast[T](id) with Logging with Serializable {
@ -64,6 +65,10 @@ private object HttpBroadcast extends Logging {
private var serverUri: String = null
private var server: HttpServer = null
private val files = new TimeStampedHashSet[String]
private val cleaner = new MetadataCleaner("HttpBroadcast", cleanup)
def initialize(isMaster: Boolean) {
synchronized {
if (!initialized) {
@ -85,6 +90,7 @@ private object HttpBroadcast extends Logging {
server = null
}
initialized = false
cleaner.cancel()
}
}
@ -108,6 +114,7 @@ private object HttpBroadcast extends Logging {
val serOut = ser.serializeStream(out)
serOut.writeObject(value)
serOut.close()
files += file.getAbsolutePath
}
def read[T](id: Long): T = {
@ -123,4 +130,21 @@ private object HttpBroadcast extends Logging {
serIn.close()
obj
}
def cleanup(cleanupTime: Long) {
val iterator = files.internalMap.entrySet().iterator()
while(iterator.hasNext) {
val entry = iterator.next()
val (file, time) = (entry.getKey, entry.getValue)
if (time < cleanupTime) {
try {
iterator.remove()
new File(file.toString).delete()
logInfo("Deleted broadcast file '" + file + "'")
} catch {
case e: Exception => logWarning("Could not delete broadcast file '" + file + "'", e)
}
}
}
}
}

View file

@ -4,7 +4,6 @@ import spark.deploy.ExecutorState.ExecutorState
import spark.deploy.master.{WorkerInfo, JobInfo}
import spark.deploy.worker.ExecutorRunner
import scala.collection.immutable.List
import scala.collection.mutable.HashMap
private[spark] sealed trait DeployMessage extends Serializable
@ -42,7 +41,8 @@ private[spark] case class LaunchExecutor(
execId: Int,
jobDesc: JobDescription,
cores: Int,
memory: Int)
memory: Int,
sparkHome: String)
extends DeployMessage

View file

@ -4,7 +4,8 @@ private[spark] class JobDescription(
val name: String,
val cores: Int,
val memoryPerSlave: Int,
val command: Command)
val command: Command,
val sparkHome: String)
extends Serializable {
val user = System.getProperty("user.name", "<unknown>")

View file

@ -25,7 +25,7 @@ private[spark] object TestClient {
val url = args(0)
val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0)
val desc = new JobDescription(
"TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()))
"TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home")
val listener = new TestListener
val client = new Client(actorSystem, url, desc, listener)
client.start()

View file

@ -173,7 +173,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
for (pos <- 0 until numUsable) {
if (assigned(pos) > 0) {
val exec = job.addExecutor(usableWorkers(pos), assigned(pos))
launchExecutor(usableWorkers(pos), exec)
launchExecutor(usableWorkers(pos), exec, job.desc.sparkHome)
job.state = JobState.RUNNING
}
}
@ -186,7 +186,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
val coresToUse = math.min(worker.coresFree, job.coresLeft)
if (coresToUse > 0) {
val exec = job.addExecutor(worker, coresToUse)
launchExecutor(worker, exec)
launchExecutor(worker, exec, job.desc.sparkHome)
job.state = JobState.RUNNING
}
}
@ -195,10 +195,10 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
}
}
def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo) {
def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) {
logInfo("Launching executor " + exec.fullId + " on worker " + worker.id)
worker.addExecutor(exec)
worker.actor ! LaunchExecutor(exec.job.id, exec.id, exec.job.desc, exec.cores, exec.memory)
worker.actor ! LaunchExecutor(exec.job.id, exec.id, exec.job.desc, exec.cores, exec.memory, sparkHome)
exec.job.actor ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory)
}

View file

@ -119,10 +119,10 @@ private[spark] class Worker(
logError("Worker registration failed: " + message)
System.exit(1)
case LaunchExecutor(jobId, execId, jobDesc, cores_, memory_) =>
case LaunchExecutor(jobId, execId, jobDesc, cores_, memory_, execSparkHome_) =>
logInfo("Asked to launch executor %s/%d for %s".format(jobId, execId, jobDesc.name))
val manager = new ExecutorRunner(
jobId, execId, jobDesc, cores_, memory_, self, workerId, ip, sparkHome, workDir)
jobId, execId, jobDesc, cores_, memory_, self, workerId, ip, new File(execSparkHome_), workDir)
executors(jobId + "/" + execId) = manager
manager.start()
coresUsed += cores_

View file

@ -1,9 +1,7 @@
package spark.rdd
import scala.collection.mutable.HashMap
import spark.{Dependency, RDD, SparkContext, SparkEnv, Split, TaskContext}
import spark.{RDD, SparkContext, SparkEnv, Split, TaskContext}
private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split {
val index = idx
@ -11,10 +9,10 @@ private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split
private[spark]
class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String])
extends RDD[T](sc) {
extends RDD[T](sc, Nil) {
@transient
val splits_ = (0 until blockIds.size).map(i => {
var splits_ : Array[Split] = (0 until blockIds.size).map(i => {
new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split]
}).toArray
@ -26,7 +24,7 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St
HashMap(blockIds.zip(locations):_*)
}
override def splits = splits_
override def getSplits = splits_
override def compute(split: Split, context: TaskContext): Iterator[T] = {
val blockManager = SparkEnv.get.blockManager
@ -38,9 +36,11 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St
}
}
override def preferredLocations(split: Split) =
override def getPreferredLocations(split: Split) =
locations_(split.asInstanceOf[BlockRDDSplit].blockId)
override val dependencies: List[Dependency[_]] = Nil
override def clearDependencies() {
splits_ = null
}
}

View file

@ -1,37 +1,54 @@
package spark.rdd
import spark.{NarrowDependency, RDD, SparkContext, Split, TaskContext}
import java.io.{ObjectOutputStream, IOException}
import spark.{OneToOneDependency, NarrowDependency, RDD, SparkContext, Split, TaskContext}
private[spark]
class CartesianSplit(idx: Int, val s1: Split, val s2: Split) extends Split with Serializable {
class CartesianSplit(
idx: Int,
@transient rdd1: RDD[_],
@transient rdd2: RDD[_],
s1Index: Int,
s2Index: Int
) extends Split {
var s1 = rdd1.splits(s1Index)
var s2 = rdd2.splits(s2Index)
override val index: Int = idx
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
s1 = rdd1.splits(s1Index)
s2 = rdd2.splits(s2Index)
oos.defaultWriteObject()
}
}
private[spark]
class CartesianRDD[T: ClassManifest, U:ClassManifest](
sc: SparkContext,
rdd1: RDD[T],
rdd2: RDD[U])
extends RDD[Pair[T, U]](sc)
var rdd1 : RDD[T],
var rdd2 : RDD[U])
extends RDD[Pair[T, U]](sc, Nil)
with Serializable {
val numSplitsInRdd2 = rdd2.splits.size
@transient
val splits_ = {
var splits_ = {
// create the cross product split
val array = new Array[Split](rdd1.splits.size * rdd2.splits.size)
for (s1 <- rdd1.splits; s2 <- rdd2.splits) {
val idx = s1.index * numSplitsInRdd2 + s2.index
array(idx) = new CartesianSplit(idx, s1, s2)
array(idx) = new CartesianSplit(idx, rdd1, rdd2, s1.index, s2.index)
}
array
}
override def splits = splits_
override def getSplits = splits_
override def preferredLocations(split: Split) = {
override def getPreferredLocations(split: Split) = {
val currSplit = split.asInstanceOf[CartesianSplit]
rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)
}
@ -42,7 +59,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest](
y <- rdd2.iterator(currSplit.s2, context)) yield (x, y)
}
override val dependencies = List(
var deps_ = List(
new NarrowDependency(rdd1) {
def getParents(id: Int): Seq[Int] = List(id / numSplitsInRdd2)
},
@ -50,4 +67,13 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest](
def getParents(id: Int): Seq[Int] = List(id % numSplitsInRdd2)
}
)
override def getDependencies = deps_
override def clearDependencies() {
deps_ = Nil
splits_ = null
rdd1 = null
rdd2 = null
}
}

View file

@ -0,0 +1,128 @@
package spark.rdd
import spark._
import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.io.{NullWritable, BytesWritable}
import org.apache.hadoop.util.ReflectionUtils
import org.apache.hadoop.fs.Path
import java.io.{File, IOException, EOFException}
import java.text.NumberFormat
private[spark] class CheckpointRDDSplit(idx: Int, val splitFile: String) extends Split {
override val index: Int = idx
}
/**
* This RDD represents a RDD checkpoint file (similar to HadoopRDD).
*/
private[spark]
class CheckpointRDD[T: ClassManifest](sc: SparkContext, checkpointPath: String)
extends RDD[T](sc, Nil) {
@transient val path = new Path(checkpointPath)
@transient val fs = path.getFileSystem(new Configuration())
@transient val splits_ : Array[Split] = {
val splitFiles = fs.listStatus(path).map(_.getPath.toString).filter(_.contains("part-")).sorted
splitFiles.zipWithIndex.map(x => new CheckpointRDDSplit(x._2, x._1)).toArray
}
checkpointData = Some(new RDDCheckpointData[T](this))
checkpointData.get.cpFile = Some(checkpointPath)
override def getSplits = splits_
override def getPreferredLocations(split: Split): Seq[String] = {
val status = fs.getFileStatus(path)
val locations = fs.getFileBlockLocations(status, 0, status.getLen)
locations.firstOption.toList.flatMap(_.getHosts).filter(_ != "localhost")
}
override def compute(split: Split, context: TaskContext): Iterator[T] = {
CheckpointRDD.readFromFile(split.asInstanceOf[CheckpointRDDSplit].splitFile, context)
}
override def checkpoint() {
// Do nothing. Hadoop RDD should not be checkpointed.
}
}
private[spark] object CheckpointRDD extends Logging {
def splitIdToFileName(splitId: Int): String = {
val numfmt = NumberFormat.getInstance()
numfmt.setMinimumIntegerDigits(5)
numfmt.setGroupingUsed(false)
"part-" + numfmt.format(splitId)
}
def writeToFile[T](path: String, blockSize: Int = -1)(context: TaskContext, iterator: Iterator[T]) {
val outputDir = new Path(path)
val fs = outputDir.getFileSystem(new Configuration())
val finalOutputName = splitIdToFileName(context.splitId)
val finalOutputPath = new Path(outputDir, finalOutputName)
val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + context.attemptId)
if (fs.exists(tempOutputPath)) {
throw new IOException("Checkpoint failed: temporary path " +
tempOutputPath + " already exists")
}
val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
val fileOutputStream = if (blockSize < 0) {
fs.create(tempOutputPath, false, bufferSize)
} else {
// This is mainly for testing purpose
fs.create(tempOutputPath, false, bufferSize, fs.getDefaultReplication, blockSize)
}
val serializer = SparkEnv.get.serializer.newInstance()
val serializeStream = serializer.serializeStream(fileOutputStream)
serializeStream.writeAll(iterator)
serializeStream.close()
if (!fs.rename(tempOutputPath, finalOutputPath)) {
if (!fs.delete(finalOutputPath, true)) {
throw new IOException("Checkpoint failed: failed to delete earlier output of task "
+ context.attemptId)
}
if (!fs.rename(tempOutputPath, finalOutputPath)) {
throw new IOException("Checkpoint failed: failed to save output of task: "
+ context.attemptId)
}
}
}
def readFromFile[T](path: String, context: TaskContext): Iterator[T] = {
val inputPath = new Path(path)
val fs = inputPath.getFileSystem(new Configuration())
val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
val fileInputStream = fs.open(inputPath, bufferSize)
val serializer = SparkEnv.get.serializer.newInstance()
val deserializeStream = serializer.deserializeStream(fileInputStream)
// Register an on-task-completion callback to close the input stream.
context.addOnCompleteCallback(() => deserializeStream.close())
deserializeStream.asIterator.asInstanceOf[Iterator[T]]
}
// Test whether CheckpointRDD generate expected number of splits despite
// each split file having multiple blocks. This needs to be run on a
// cluster (mesos or standalone) using HDFS.
def main(args: Array[String]) {
import spark._
val Array(cluster, hdfsPath) = args
val sc = new SparkContext(cluster, "CheckpointRDD Test")
val rdd = sc.makeRDD(1 to 10, 10).flatMap(x => 1 to 10000)
val path = new Path(hdfsPath, "temp")
val fs = path.getFileSystem(new Configuration())
sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _)
val cpRDD = new CheckpointRDD[Int](sc, path.toString)
assert(cpRDD.splits.length == rdd.splits.length, "Number of splits is not the same")
assert(cpRDD.collect.toList == rdd.collect.toList, "Data of splits not the same")
fs.delete(path)
}
}

View file

@ -1,5 +1,6 @@
package spark.rdd
import java.io.{ObjectOutputStream, IOException}
import java.util.{HashMap => JHashMap}
import scala.collection.JavaConversions
import scala.collection.mutable.ArrayBuffer
@ -9,7 +10,21 @@ import spark.{Dependency, OneToOneDependency, ShuffleDependency}
private[spark] sealed trait CoGroupSplitDep extends Serializable
private[spark] case class NarrowCoGroupSplitDep(rdd: RDD[_], split: Split) extends CoGroupSplitDep
private[spark] case class NarrowCoGroupSplitDep(
rdd: RDD[_],
splitIndex: Int,
var split: Split
) extends CoGroupSplitDep {
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
split = rdd.splits(splitIndex)
oos.defaultWriteObject()
}
}
private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
private[spark]
@ -25,30 +40,31 @@ private[spark] class CoGroupAggregator
{ (b1, b2) => b1 ++ b2 })
with Serializable
class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
extends RDD[(K, Seq[Seq[_]])](rdds.head.context) with Logging {
class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner)
extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) with Logging {
val aggr = new CoGroupAggregator
@transient
override val dependencies = {
var deps_ = {
val deps = new ArrayBuffer[Dependency[_]]
for ((rdd, index) <- rdds.zipWithIndex) {
val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true)
if (mapSideCombinedRDD.partitioner == Some(part)) {
logInfo("Adding one-to-one dependency with " + mapSideCombinedRDD)
deps += new OneToOneDependency(mapSideCombinedRDD)
if (rdd.partitioner == Some(part)) {
logInfo("Adding one-to-one dependency with " + rdd)
deps += new OneToOneDependency(rdd)
} else {
logInfo("Adding shuffle dependency with " + rdd)
val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true)
deps += new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part)
}
}
deps.toList
}
override def getDependencies = deps_
@transient
val splits_ : Array[Split] = {
val firstRdd = rdds.head
var splits_ : Array[Split] = {
val array = new Array[Split](part.numPartitions)
for (i <- 0 until array.size) {
array(i) = new CoGroupSplit(i, rdds.zipWithIndex.map { case (r, j) =>
@ -56,19 +72,17 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
case s: ShuffleDependency[_, _] =>
new ShuffleCoGroupSplitDep(s.shuffleId): CoGroupSplitDep
case _ =>
new NarrowCoGroupSplitDep(r, r.splits(i)): CoGroupSplitDep
new NarrowCoGroupSplitDep(r, i, r.splits(i)): CoGroupSplitDep
}
}.toList)
}
array
}
override def splits = splits_
override def getSplits = splits_
override val partitioner = Some(part)
override def preferredLocations(s: Split) = Nil
override def compute(s: Split, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = {
val split = s.asInstanceOf[CoGroupSplit]
val numRdds = split.deps.size
@ -84,7 +98,7 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
}
}
for ((dep, depNum) <- split.deps.zipWithIndex) dep match {
case NarrowCoGroupSplitDep(rdd, itsSplit) => {
case NarrowCoGroupSplitDep(rdd, itsSplitIndex, itsSplit) => {
// Read them from the parent
for ((k, v) <- rdd.iterator(itsSplit, context)) {
getSeq(k.asInstanceOf[K])(depNum) += v
@ -103,4 +117,10 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
}
JavaConversions.mapAsScalaMap(map).iterator
}
override def clearDependencies() {
deps_ = null
splits_ = null
rdds = null
}
}

View file

@ -1,9 +1,22 @@
package spark.rdd
import spark.{NarrowDependency, RDD, Split, TaskContext}
import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Split, TaskContext}
import java.io.{ObjectOutputStream, IOException}
private[spark] case class CoalescedRDDSplit(
index: Int,
@transient rdd: RDD[_],
parentsIndices: Array[Int]
) extends Split {
var parents: Seq[Split] = parentsIndices.map(rdd.splits(_))
private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) extends Split
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
parents = parentsIndices.map(rdd.splits(_))
oos.defaultWriteObject()
}
}
/**
* Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of
@ -13,34 +26,44 @@ private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) exten
* This transformation is useful when an RDD with many partitions gets filtered into a smaller one,
* or to avoid having a large number of small tasks when processing a directory with many files.
*/
class CoalescedRDD[T: ClassManifest](prev: RDD[T], maxPartitions: Int)
extends RDD[T](prev.context) {
class CoalescedRDD[T: ClassManifest](
var prev: RDD[T],
maxPartitions: Int)
extends RDD[T](prev.context, Nil) { // Nil, so the dependencies_ var does not refer to parent RDDs
@transient val splits_ : Array[Split] = {
@transient var splits_ : Array[Split] = {
val prevSplits = prev.splits
if (prevSplits.length < maxPartitions) {
prevSplits.zipWithIndex.map{ case (s, idx) => new CoalescedRDDSplit(idx, Array(s)) }
prevSplits.map(_.index).map{idx => new CoalescedRDDSplit(idx, prev, Array(idx)) }
} else {
(0 until maxPartitions).map { i =>
val rangeStart = (i * prevSplits.length) / maxPartitions
val rangeEnd = ((i + 1) * prevSplits.length) / maxPartitions
new CoalescedRDDSplit(i, prevSplits.slice(rangeStart, rangeEnd))
new CoalescedRDDSplit(i, prev, (rangeStart until rangeEnd).toArray)
}.toArray
}
}
override def splits = splits_
override def getSplits = splits_
override def compute(split: Split, context: TaskContext): Iterator[T] = {
split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap {
parentSplit => prev.iterator(parentSplit, context)
split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap { parentSplit =>
firstParent[T].iterator(parentSplit, context)
}
}
val dependencies = List(
var deps_ : List[Dependency[_]] = List(
new NarrowDependency(prev) {
def getParents(id: Int): Seq[Int] =
splits(id).asInstanceOf[CoalescedRDDSplit].parents.map(_.index)
splits(id).asInstanceOf[CoalescedRDDSplit].parentsIndices
}
)
override def getDependencies() = deps_
override def clearDependencies() {
deps_ = Nil
splits_ = null
prev = null
}
}

View file

@ -2,11 +2,15 @@ package spark.rdd
import spark.{OneToOneDependency, RDD, Split, TaskContext}
private[spark] class FilteredRDD[T: ClassManifest](
prev: RDD[T],
f: T => Boolean)
extends RDD[T](prev) {
override def getSplits = firstParent[T].splits
private[spark]
class FilteredRDD[T: ClassManifest](prev: RDD[T], f: T => Boolean) extends RDD[T](prev.context) {
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
override val partitioner = prev.partitioner // Since filter cannot change a partition's keys
override def compute(split: Split, context: TaskContext) = prev.iterator(split, context).filter(f)
override def compute(split: Split, context: TaskContext) =
firstParent[T].iterator(split, context).filter(f)
}

View file

@ -1,16 +1,16 @@
package spark.rdd
import spark.{OneToOneDependency, RDD, Split, TaskContext}
import spark.{RDD, Split, TaskContext}
private[spark]
class FlatMappedRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
f: T => TraversableOnce[U])
extends RDD[U](prev.context) {
extends RDD[U](prev) {
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
override def getSplits = firstParent[T].splits
override def compute(split: Split, context: TaskContext) =
prev.iterator(split, context).flatMap(f)
firstParent[T].iterator(split, context).flatMap(f)
}

View file

@ -1,12 +1,12 @@
package spark.rdd
import spark.{OneToOneDependency, RDD, Split, TaskContext}
import spark.{RDD, Split, TaskContext}
private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T])
extends RDD[Array[T]](prev) {
override def getSplits = firstParent[T].splits
private[spark]
class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev.context) {
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
override def compute(split: Split, context: TaskContext) =
Array(prev.iterator(split, context).toArray).iterator
}
Array(firstParent[T].iterator(split, context).toArray).iterator
}

View file

@ -22,9 +22,8 @@ import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskCo
* A Spark split class that wraps around a Hadoop InputSplit.
*/
private[spark] class HadoopSplit(rddId: Int, idx: Int, @transient s: InputSplit)
extends Split
with Serializable {
extends Split {
val inputSplit = new SerializableWritable[InputSplit](s)
override def hashCode(): Int = (41 * (41 + rddId) + idx).toInt
@ -43,7 +42,7 @@ class HadoopRDD[K, V](
keyClass: Class[K],
valueClass: Class[V],
minSplits: Int)
extends RDD[(K, V)](sc) {
extends RDD[(K, V)](sc, Nil) {
// A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it
val confBroadcast = sc.broadcast(new SerializableWritable(conf))
@ -64,7 +63,7 @@ class HadoopRDD[K, V](
.asInstanceOf[InputFormat[K, V]]
}
override def splits = splits_
override def getSplits = splits_
override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] {
val split = theSplit.asInstanceOf[HadoopSplit]
@ -110,11 +109,13 @@ class HadoopRDD[K, V](
}
}
override def preferredLocations(split: Split) = {
override def getPreferredLocations(split: Split) = {
// TODO: Filtering out "localhost" in case of file:// URLs
val hadoopSplit = split.asInstanceOf[HadoopSplit]
hadoopSplit.inputSplit.value.getLocations.filter(_ != "localhost")
}
override val dependencies: List[Dependency[_]] = Nil
override def checkpoint() {
// Do nothing. Hadoop RDD should not be checkpointed.
}
}

View file

@ -1,6 +1,6 @@
package spark.rdd
import spark.{OneToOneDependency, RDD, Split, TaskContext}
import spark.{RDD, Split, TaskContext}
private[spark]
@ -8,11 +8,13 @@ class MapPartitionsRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
f: Iterator[T] => Iterator[U],
preservesPartitioning: Boolean = false)
extends RDD[U](prev.context) {
extends RDD[U](prev) {
override val partitioner = if (preservesPartitioning) prev.partitioner else None
override val partitioner =
if (preservesPartitioning) firstParent[T].partitioner else None
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
override def compute(split: Split, context: TaskContext) = f(prev.iterator(split, context))
override def getSplits = firstParent[T].splits
override def compute(split: Split, context: TaskContext) =
f(firstParent[T].iterator(split, context))
}

View file

@ -1,6 +1,7 @@
package spark.rdd
import spark.{OneToOneDependency, RDD, Split, TaskContext}
import spark.{RDD, Split, TaskContext}
/**
* A variant of the MapPartitionsRDD that passes the split index into the
@ -11,12 +12,13 @@ private[spark]
class MapPartitionsWithSplitRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
f: (Int, Iterator[T]) => Iterator[U],
preservesPartitioning: Boolean)
extends RDD[U](prev.context) {
preservesPartitioning: Boolean
) extends RDD[U](prev) {
override def getSplits = firstParent[T].splits
override val partitioner = if (preservesPartitioning) prev.partitioner else None
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
override def compute(split: Split, context: TaskContext) =
f(split.index, prev.iterator(split, context))
f(split.index, firstParent[T].iterator(split, context))
}

View file

@ -1,14 +1,15 @@
package spark.rdd
import spark.{OneToOneDependency, RDD, Split, TaskContext}
import spark.{RDD, Split, TaskContext}
private[spark]
class MappedRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
f: T => U)
extends RDD[U](prev.context) {
extends RDD[U](prev) {
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
override def compute(split: Split, context: TaskContext) = prev.iterator(split, context).map(f)
override def getSplits = firstParent[T].splits
override def compute(split: Split, context: TaskContext) =
firstParent[T].iterator(split, context).map(f)
}

View file

@ -20,11 +20,12 @@ class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit
}
class NewHadoopRDD[K, V](
sc: SparkContext,
sc : SparkContext,
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K], valueClass: Class[V],
keyClass: Class[K],
valueClass: Class[V],
@transient conf: Configuration)
extends RDD[(K, V)](sc)
extends RDD[(K, V)](sc, Nil)
with HadoopMapReduceUtil {
// A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
@ -51,7 +52,7 @@ class NewHadoopRDD[K, V](
result
}
override def splits = splits_
override def getSplits = splits_
override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] {
val split = theSplit.asInstanceOf[NewHadoopSplit]
@ -86,10 +87,8 @@ class NewHadoopRDD[K, V](
}
}
override def preferredLocations(split: Split) = {
override def getPreferredLocations(split: Split) = {
val theSplit = split.asInstanceOf[NewHadoopSplit]
theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost")
}
override val dependencies: List[Dependency[_]] = Nil
}

View file

@ -8,7 +8,7 @@ import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.io.Source
import spark.{OneToOneDependency, RDD, SparkEnv, Split, TaskContext}
import spark.{RDD, SparkEnv, Split, TaskContext}
/**
@ -16,18 +16,18 @@ import spark.{OneToOneDependency, RDD, SparkEnv, Split, TaskContext}
* (printing them one per line) and returns the output as a collection of strings.
*/
class PipedRDD[T: ClassManifest](
parent: RDD[T], command: Seq[String], envVars: Map[String, String])
extends RDD[String](parent.context) {
prev: RDD[T],
command: Seq[String],
envVars: Map[String, String])
extends RDD[String](prev) {
def this(parent: RDD[T], command: Seq[String]) = this(parent, command, Map())
def this(prev: RDD[T], command: Seq[String]) = this(prev, command, Map())
// Similar to Runtime.exec(), if we are given a single string, split it into words
// using a standard StringTokenizer (i.e. by spaces)
def this(parent: RDD[T], command: String) = this(parent, PipedRDD.tokenize(command))
def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command))
override def splits = parent.splits
override val dependencies = List(new OneToOneDependency(parent))
override def getSplits = firstParent[T].splits
override def compute(split: Split, context: TaskContext): Iterator[String] = {
val pb = new ProcessBuilder(command)
@ -52,7 +52,7 @@ class PipedRDD[T: ClassManifest](
override def run() {
SparkEnv.set(env)
val out = new PrintWriter(proc.getOutputStream)
for (elem <- parent.iterator(split, context)) {
for (elem <- firstParent[T].iterator(split, context)) {
out.println(elem)
}
out.close()

View file

@ -1,11 +1,11 @@
package spark.rdd
import java.util.Random
import cern.jet.random.Poisson
import cern.jet.random.engine.DRand
import spark.{OneToOneDependency, RDD, Split, TaskContext}
import spark.{RDD, Split, TaskContext}
private[spark]
class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Serializable {
@ -14,23 +14,21 @@ class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Seriali
class SampledRDD[T: ClassManifest](
prev: RDD[T],
withReplacement: Boolean,
withReplacement: Boolean,
frac: Double,
seed: Int)
extends RDD[T](prev.context) {
extends RDD[T](prev) {
@transient
val splits_ = {
var splits_ : Array[Split] = {
val rg = new Random(seed)
prev.splits.map(x => new SampledRDDSplit(x, rg.nextInt))
firstParent[T].splits.map(x => new SampledRDDSplit(x, rg.nextInt))
}
override def splits = splits_.asInstanceOf[Array[Split]]
override def getSplits = splits_.asInstanceOf[Array[Split]]
override val dependencies = List(new OneToOneDependency(prev))
override def preferredLocations(split: Split) =
prev.preferredLocations(split.asInstanceOf[SampledRDDSplit].prev)
override def getPreferredLocations(split: Split) =
firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDSplit].prev)
override def compute(splitIn: Split, context: TaskContext) = {
val split = splitIn.asInstanceOf[SampledRDDSplit]
@ -38,7 +36,7 @@ class SampledRDD[T: ClassManifest](
// For large datasets, the expected number of occurrences of each element in a sample with
// replacement is Poisson(frac). We use that to get a count for each element.
val poisson = new Poisson(frac, new DRand(split.seed))
prev.iterator(split.prev, context).flatMap { element =>
firstParent[T].iterator(split.prev, context).flatMap { element =>
val count = poisson.nextInt()
if (count == 0) {
Iterator.empty // Avoid object allocation when we return 0 items, which is quite often
@ -48,7 +46,11 @@ class SampledRDD[T: ClassManifest](
}
} else { // Sampling without replacement
val rand = new Random(split.seed)
prev.iterator(split.prev, context).filter(x => (rand.nextDouble <= frac))
firstParent[T].iterator(split.prev, context).filter(x => (rand.nextDouble <= frac))
}
}
override def clearDependencies() {
splits_ = null
}
}

View file

@ -1,7 +1,7 @@
package spark.rdd
import spark.{OneToOneDependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Split, TaskContext}
import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Split, TaskContext}
import spark.SparkContext._
private[spark] class ShuffledRDDSplit(val idx: Int) extends Split {
override val index = idx
@ -10,28 +10,29 @@ private[spark] class ShuffledRDDSplit(val idx: Int) extends Split {
/**
* The resulting RDD from a shuffle (e.g. repartitioning of data).
* @param parent the parent RDD.
* @param prev the parent RDD.
* @param part the partitioner used to partition the RDD
* @tparam K the key class.
* @tparam V the value class.
*/
class ShuffledRDD[K, V](
@transient parent: RDD[(K, V)],
part: Partitioner) extends RDD[(K, V)](parent.context) {
prev: RDD[(K, V)],
part: Partitioner)
extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part))) {
override val partitioner = Some(part)
@transient
val splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
var splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
override def splits = splits_
override def preferredLocations(split: Split) = Nil
val dep = new ShuffleDependency(parent, part)
override val dependencies = List(dep)
override def getSplits = splits_
override def compute(split: Split, context: TaskContext): Iterator[(K, V)] = {
SparkEnv.get.shuffleFetcher.fetch[K, V](dep.shuffleId, split.index)
val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId
SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index)
}
override def clearDependencies() {
splits_ = null
}
}

View file

@ -1,43 +1,47 @@
package spark.rdd
import scala.collection.mutable.ArrayBuffer
import spark.{Dependency, RangeDependency, RDD, SparkContext, Split, TaskContext}
import java.io.{ObjectOutputStream, IOException}
private[spark] class UnionSplit[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int)
extends Split {
private[spark] class UnionSplit[T: ClassManifest](
idx: Int,
rdd: RDD[T],
split: Split)
extends Split
with Serializable {
var split: Split = rdd.splits(splitIndex)
def iterator(context: TaskContext) = rdd.iterator(split, context)
def preferredLocations() = rdd.preferredLocations(split)
override val index: Int = idx
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
split = rdd.splits(splitIndex)
oos.defaultWriteObject()
}
}
class UnionRDD[T: ClassManifest](
sc: SparkContext,
@transient rdds: Seq[RDD[T]])
extends RDD[T](sc)
with Serializable {
@transient var rdds: Seq[RDD[T]])
extends RDD[T](sc, Nil) { // Nil, so the dependencies_ var does not refer to parent RDDs
@transient
val splits_ : Array[Split] = {
var splits_ : Array[Split] = {
val array = new Array[Split](rdds.map(_.splits.size).sum)
var pos = 0
for (rdd <- rdds; split <- rdd.splits) {
array(pos) = new UnionSplit(pos, rdd, split)
array(pos) = new UnionSplit(pos, rdd, split.index)
pos += 1
}
array
}
override def splits = splits_
override def getSplits = splits_
@transient
override val dependencies = {
@transient var deps_ = {
val deps = new ArrayBuffer[Dependency[_]]
var pos = 0
for (rdd <- rdds) {
@ -47,9 +51,17 @@ class UnionRDD[T: ClassManifest](
deps.toList
}
override def getDependencies = deps_
override def compute(s: Split, context: TaskContext): Iterator[T] =
s.asInstanceOf[UnionSplit[T]].iterator(context)
override def preferredLocations(s: Split): Seq[String] =
override def getPreferredLocations(s: Split): Seq[String] =
s.asInstanceOf[UnionSplit[T]].preferredLocations()
override def clearDependencies() {
deps_ = null
splits_ = null
rdds = null
}
}

View file

@ -1,53 +1,66 @@
package spark.rdd
import spark.{OneToOneDependency, RDD, SparkContext, Split, TaskContext}
import java.io.{ObjectOutputStream, IOException}
private[spark] class ZippedSplit[T: ClassManifest, U: ClassManifest](
idx: Int,
rdd1: RDD[T],
rdd2: RDD[U],
split1: Split,
split2: Split)
extends Split
with Serializable {
def iterator(context: TaskContext): Iterator[(T, U)] =
rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context))
def preferredLocations(): Seq[String] =
rdd1.preferredLocations(split1).intersect(rdd2.preferredLocations(split2))
@transient rdd1: RDD[T],
@transient rdd2: RDD[U]
) extends Split {
var split1 = rdd1.splits(idx)
var split2 = rdd1.splits(idx)
override val index: Int = idx
def splits = (split1, split2)
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
split1 = rdd1.splits(idx)
split2 = rdd2.splits(idx)
oos.defaultWriteObject()
}
}
class ZippedRDD[T: ClassManifest, U: ClassManifest](
sc: SparkContext,
@transient rdd1: RDD[T],
@transient rdd2: RDD[U])
extends RDD[(T, U)](sc)
var rdd1: RDD[T],
var rdd2: RDD[U])
extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2)))
with Serializable {
// TODO: FIX THIS.
@transient
val splits_ : Array[Split] = {
var splits_ : Array[Split] = {
if (rdd1.splits.size != rdd2.splits.size) {
throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions")
}
val array = new Array[Split](rdd1.splits.size)
for (i <- 0 until rdd1.splits.size) {
array(i) = new ZippedSplit(i, rdd1, rdd2, rdd1.splits(i), rdd2.splits(i))
array(i) = new ZippedSplit(i, rdd1, rdd2)
}
array
}
override def splits = splits_
override def getSplits = splits_
@transient
override val dependencies = List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))
override def compute(s: Split, context: TaskContext): Iterator[(T, U)] = {
val (split1, split2) = s.asInstanceOf[ZippedSplit[T, U]].splits
rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context))
}
override def compute(s: Split, context: TaskContext): Iterator[(T, U)] =
s.asInstanceOf[ZippedSplit[T, U]].iterator(context)
override def getPreferredLocations(s: Split): Seq[String] = {
val (split1, split2) = s.asInstanceOf[ZippedSplit[T, U]].splits
rdd1.preferredLocations(split1).intersect(rdd2.preferredLocations(split2))
}
override def preferredLocations(s: Split): Seq[String] =
s.asInstanceOf[ZippedSplit[T, U]].preferredLocations()
override def clearDependencies() {
splits_ = null
rdd1 = null
rdd2 = null
}
}

View file

@ -14,6 +14,7 @@ import spark.partial.ApproximateEvaluator
import spark.partial.PartialResult
import spark.storage.BlockManagerMaster
import spark.storage.BlockManagerId
import util.{MetadataCleaner, TimeStampedHashMap}
/**
* A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for
@ -61,9 +62,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
val nextStageId = new AtomicInteger(0)
val idToStage = new HashMap[Int, Stage]
val idToStage = new TimeStampedHashMap[Int, Stage]
val shuffleToMapStage = new HashMap[Int, Stage]
val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
var cacheLocs = new HashMap[Int, Array[List[String]]]
@ -77,12 +78,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done
val running = new HashSet[Stage] // Stages we are running right now
val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures
val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage
val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage
var lastFetchFailureTime: Long = 0 // Used to wait a bit to avoid repeated resubmits
val activeJobs = new HashSet[ActiveJob]
val resultStageToJob = new HashMap[Stage, ActiveJob]
val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup)
// Start a thread to run the DAGScheduler event loop
new Thread("DAGScheduler") {
setDaemon(true)
@ -594,8 +597,23 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
return Nil
}
def cleanup(cleanupTime: Long) {
var sizeBefore = idToStage.size
idToStage.clearOldValues(cleanupTime)
logInfo("idToStage " + sizeBefore + " --> " + idToStage.size)
sizeBefore = shuffleToMapStage.size
shuffleToMapStage.clearOldValues(cleanupTime)
logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size)
sizeBefore = pendingTasks.size
pendingTasks.clearOldValues(cleanupTime)
logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size)
}
def stop() {
eventQueue.put(StopDAGScheduler)
metadataCleaner.cancel()
taskSched.stop()
}
}

View file

@ -1,17 +1,74 @@
package spark.scheduler
import spark._
import java.io._
import util.{MetadataCleaner, TimeStampedHashMap}
import java.util.zip.{GZIPInputStream, GZIPOutputStream}
private[spark] object ResultTask {
// A simple map between the stage id to the serialized byte array of a task.
// Served as a cache for task serialization because serialization can be
// expensive on the master node if it needs to launch thousands of tasks.
val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
val metadataCleaner = new MetadataCleaner("ResultTask", serializedInfoCache.clearOldValues)
def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = {
synchronized {
val old = serializedInfoCache.get(stageId).orNull
if (old != null) {
return old
} else {
val out = new ByteArrayOutputStream
val ser = SparkEnv.get.closureSerializer.newInstance
val objOut = ser.serializeStream(new GZIPOutputStream(out))
objOut.writeObject(rdd)
objOut.writeObject(func)
objOut.close()
val bytes = out.toByteArray
serializedInfoCache.put(stageId, bytes)
return bytes
}
}
}
def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = {
synchronized {
val loader = Thread.currentThread.getContextClassLoader
val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
val ser = SparkEnv.get.closureSerializer.newInstance
val objIn = ser.deserializeStream(in)
val rdd = objIn.readObject().asInstanceOf[RDD[_]]
val func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _]
return (rdd, func)
}
}
def clearCache() {
synchronized {
serializedInfoCache.clear()
}
}
}
private[spark] class ResultTask[T, U](
stageId: Int,
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
val partition: Int,
var rdd: RDD[T],
var func: (TaskContext, Iterator[T]) => U,
var partition: Int,
@transient locs: Seq[String],
val outputId: Int)
extends Task[U](stageId) {
extends Task[U](stageId) with Externalizable {
val split = rdd.splits(partition)
def this() = this(0, null, null, 0, null, 0)
var split = if (rdd == null) {
null
} else {
rdd.splits(partition)
}
override def run(attemptId: Long): U = {
val context = new TaskContext(stageId, partition, attemptId)
@ -25,4 +82,31 @@ private[spark] class ResultTask[T, U](
override def preferredLocations: Seq[String] = locs
override def toString = "ResultTask(" + stageId + ", " + partition + ")"
override def writeExternal(out: ObjectOutput) {
RDDCheckpointData.synchronized {
split = rdd.splits(partition)
out.writeInt(stageId)
val bytes = ResultTask.serializeInfo(
stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _])
out.writeInt(bytes.length)
out.write(bytes)
out.writeInt(partition)
out.writeInt(outputId)
out.writeObject(split)
}
}
override def readExternal(in: ObjectInput) {
val stageId = in.readInt()
val numBytes = in.readInt()
val bytes = new Array[Byte](numBytes)
in.readFully(bytes)
val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes)
rdd = rdd_.asInstanceOf[RDD[T]]
func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U]
partition = in.readInt()
val outputId = in.readInt()
split = in.readObject().asInstanceOf[Split]
}
}

View file

@ -14,17 +14,20 @@ import com.ning.compress.lzf.LZFOutputStream
import spark._
import spark.storage._
import util.{TimeStampedHashMap, MetadataCleaner}
private[spark] object ShuffleMapTask {
// A simple map between the stage id to the serialized byte array of a task.
// Served as a cache for task serialization because serialization can be
// expensive on the master node if it needs to launch thousands of tasks.
val serializedInfoCache = new JHashMap[Int, Array[Byte]]
val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
val metadataCleaner = new MetadataCleaner("ShuffleMapTask", serializedInfoCache.clearOldValues)
def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = {
synchronized {
val old = serializedInfoCache.get(stageId)
val old = serializedInfoCache.get(stageId).orNull
if (old != null) {
return old
} else {
@ -87,13 +90,16 @@ private[spark] class ShuffleMapTask(
}
override def writeExternal(out: ObjectOutput) {
out.writeInt(stageId)
val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep)
out.writeInt(bytes.length)
out.write(bytes)
out.writeInt(partition)
out.writeLong(generation)
out.writeObject(split)
RDDCheckpointData.synchronized {
split = rdd.splits(partition)
out.writeInt(stageId)
val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep)
out.writeInt(bytes.length)
out.write(bytes)
out.writeInt(partition)
out.writeLong(generation)
out.writeObject(split)
}
}
override def readExternal(in: ObjectInput) {

View file

@ -39,7 +39,8 @@ private[spark] class SparkDeploySchedulerBackend(
StandaloneSchedulerBackend.ACTOR_NAME)
val args = Seq(masterUrl, "{{SLAVEID}}", "{{HOSTNAME}}", "{{CORES}}")
val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs)
val jobDesc = new JobDescription(jobName, maxCores, executorMemory, command)
val sparkHome = sc.getSparkHome().getOrElse(throw new IllegalArgumentException("must supply spark home for spark standalone"))
val jobDesc = new JobDescription(jobName, maxCores, executorMemory, command, sparkHome)
client = new Client(sc.env.actorSystem, master, jobDesc, this)
client.start()

View file

@ -81,7 +81,10 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]](
ser.serialize(Accumulators.values))
logInfo("Finished task " + idInJob)
listener.taskEnded(task, Success, resultToReturn, accumUpdates)
// If the threadpool has not already been shutdown, notify DAGScheduler
if (!Thread.currentThread().isInterrupted)
listener.taskEnded(task, Success, resultToReturn, accumUpdates)
} catch {
case t: Throwable => {
logError("Exception in task " + idInJob, t)
@ -91,7 +94,8 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
submitTask(task, idInJob)
} else {
// TODO: Do something nicer here to return all the way to the user
listener.taskEnded(task, new ExceptionFailure(t), null, null)
if (!Thread.currentThread().isInterrupted)
listener.taskEnded(task, new ExceptionFailure(t), null, null)
}
}
}

View file

@ -17,7 +17,6 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
private val entries = new LinkedHashMap[String, Entry](32, 0.75f, true)
private var currentMemory = 0L
// Object used to ensure that only one thread is putting blocks and if necessary, dropping
// blocks from the memory store.
private val putLock = new Object()

View file

@ -32,6 +32,7 @@ private[spark] object AkkaUtils {
akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
akka.remote.transport = "akka.remote.netty.NettyRemoteTransport"
akka.remote.log-remote-lifecycle-events = on
akka.remote.netty.hostname = "%s"
akka.remote.netty.port = %d
akka.remote.netty.connection-timeout = %ds

View file

@ -4,9 +4,10 @@ import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors}
import java.util.{TimerTask, Timer}
import spark.Logging
class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging {
val delaySeconds = (System.getProperty("spark.cleanup.delay", "-100").toDouble * 60).toInt
val delaySeconds = MetadataCleaner.getDelaySeconds
val periodSeconds = math.max(10, delaySeconds / 10)
val timer = new Timer(name + " cleanup timer", true)
@ -22,6 +23,7 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging
}
}
}
if (periodSeconds > 0) {
logInfo(
"Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds and "
@ -33,3 +35,10 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging
timer.cancel()
}
}
object MetadataCleaner {
def getDelaySeconds = (System.getProperty("spark.cleaner.delay", "-100").toDouble * 60).toInt
def setDelaySeconds(delay: Long) { System.setProperty("spark.cleaner.delay", delay.toString) }
}

View file

@ -0,0 +1,62 @@
package spark.util
import scala.annotation.tailrec
import java.io.OutputStream
import java.util.concurrent.TimeUnit._
class RateLimitedOutputStream(out: OutputStream, bytesPerSec: Int) extends OutputStream {
val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS)
val CHUNK_SIZE = 8192
var lastSyncTime = System.nanoTime
var bytesWrittenSinceSync: Long = 0
override def write(b: Int) {
waitToWrite(1)
out.write(b)
}
override def write(bytes: Array[Byte]) {
write(bytes, 0, bytes.length)
}
@tailrec
override final def write(bytes: Array[Byte], offset: Int, length: Int) {
val writeSize = math.min(length - offset, CHUNK_SIZE)
if (writeSize > 0) {
waitToWrite(writeSize)
out.write(bytes, offset, writeSize)
write(bytes, offset + writeSize, length)
}
}
override def flush() {
out.flush()
}
override def close() {
out.close()
}
@tailrec
private def waitToWrite(numBytes: Int) {
val now = System.nanoTime
val elapsedSecs = SECONDS.convert(math.max(now - lastSyncTime, 1), NANOSECONDS)
val rate = bytesWrittenSinceSync.toDouble / elapsedSecs
if (rate < bytesPerSec) {
// It's okay to write; just update some variables and return
bytesWrittenSinceSync += numBytes
if (now > lastSyncTime + SYNC_INTERVAL) {
// Sync interval has passed; let's resync
lastSyncTime = now
bytesWrittenSinceSync = numBytes
}
} else {
// Calculate how much time we should sleep to bring ourselves to the desired rate.
// Based on throttler in Kafka (https://github.com/kafka-dev/kafka/blob/master/core/src/main/scala/kafka/utils/Throttler.scala)
val sleepTime = MILLISECONDS.convert((bytesWrittenSinceSync / bytesPerSec - elapsedSecs), SECONDS)
if (sleepTime > 0) Thread.sleep(sleepTime)
waitToWrite(numBytes)
}
}
}

View file

@ -1,16 +1,16 @@
package spark.util
import java.util.concurrent.ConcurrentHashMap
import scala.collection.JavaConversions._
import scala.collection.mutable.{HashMap, Map}
import scala.collection.JavaConversions
import scala.collection.mutable.Map
/**
* This is a custom implementation of scala.collection.mutable.Map which stores the insertion
* time stamp along with each key-value pair. Key-value pairs that are older than a particular
* threshold time can them be removed using the cleanup method. This is intended to be a drop-in
* threshold time can them be removed using the clearOldValues method. This is intended to be a drop-in
* replacement of scala.collection.mutable.HashMap.
*/
class TimeStampedHashMap[A, B] extends Map[A, B]() {
class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging {
val internalMap = new ConcurrentHashMap[A, (B, Long)]()
def get(key: A): Option[B] = {
@ -20,7 +20,7 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
def iterator: Iterator[(A, B)] = {
val jIterator = internalMap.entrySet().iterator()
jIterator.map(kv => (kv.getKey, kv.getValue._1))
JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue._1))
}
override def + [B1 >: B](kv: (A, B1)): Map[A, B1] = {
@ -31,8 +31,10 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
}
override def - (key: A): Map[A, B] = {
internalMap.remove(key)
this
val newMap = new TimeStampedHashMap[A, B]
newMap.internalMap.putAll(this.internalMap)
newMap.internalMap.remove(key)
newMap
}
override def += (kv: (A, B)): this.type = {
@ -56,7 +58,7 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
}
override def filter(p: ((A, B)) => Boolean): Map[A, B] = {
internalMap.map(kv => (kv._1, kv._2._1)).filter(p)
JavaConversions.asScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p)
}
override def empty: Map[A, B] = new TimeStampedHashMap[A, B]()
@ -72,11 +74,15 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
}
}
def cleanup(threshTime: Long) {
/**
* Removes old key-value pairs that have timestamp earlier than `threshTime`
*/
def clearOldValues(threshTime: Long) {
val iterator = internalMap.entrySet().iterator()
while(iterator.hasNext) {
val entry = iterator.next()
if (entry.getValue._2 < threshTime) {
logDebug("Removing key " + entry.getKey)
iterator.remove()
}
}

View file

@ -0,0 +1,69 @@
package spark.util
import scala.collection.mutable.Set
import scala.collection.JavaConversions
import java.util.concurrent.ConcurrentHashMap
class TimeStampedHashSet[A] extends Set[A] {
val internalMap = new ConcurrentHashMap[A, Long]()
def contains(key: A): Boolean = {
internalMap.contains(key)
}
def iterator: Iterator[A] = {
val jIterator = internalMap.entrySet().iterator()
JavaConversions.asScalaIterator(jIterator).map(_.getKey)
}
override def + (elem: A): Set[A] = {
val newSet = new TimeStampedHashSet[A]
newSet ++= this
newSet += elem
newSet
}
override def - (elem: A): Set[A] = {
val newSet = new TimeStampedHashSet[A]
newSet ++= this
newSet -= elem
newSet
}
override def += (key: A): this.type = {
internalMap.put(key, currentTime)
this
}
override def -= (key: A): this.type = {
internalMap.remove(key)
this
}
override def empty: Set[A] = new TimeStampedHashSet[A]()
override def size(): Int = internalMap.size()
override def foreach[U](f: (A) => U): Unit = {
val iterator = internalMap.entrySet().iterator()
while(iterator.hasNext) {
f(iterator.next.getKey)
}
}
/**
* Removes old values that have timestamp earlier than `threshTime`
*/
def clearOldValues(threshTime: Long) {
val iterator = internalMap.entrySet().iterator()
while(iterator.hasNext) {
val entry = iterator.next()
if (entry.getValue < threshTime) {
iterator.remove()
}
}
}
private def currentTime: Long = System.currentTimeMillis()
}

View file

@ -1,8 +1,8 @@
# Set everything to be logged to the console
# Set everything to be logged to the file core/target/unit-tests.log
log4j.rootCategory=INFO, file
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
log4j.appender.file.file=spark-tests.log
log4j.appender.file.file=core/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n

View file

@ -0,0 +1,357 @@
package spark
import org.scalatest.{BeforeAndAfter, FunSuite}
import java.io.File
import spark.rdd._
import spark.SparkContext._
import storage.StorageLevel
class CheckpointSuite extends FunSuite with BeforeAndAfter with Logging {
initLogging()
var sc: SparkContext = _
var checkpointDir: File = _
val partitioner = new HashPartitioner(2)
before {
checkpointDir = File.createTempFile("temp", "")
checkpointDir.delete()
sc = new SparkContext("local", "test")
sc.setCheckpointDir(checkpointDir.toString)
}
after {
if (sc != null) {
sc.stop()
sc = null
}
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.master.port")
if (checkpointDir != null) {
checkpointDir.delete()
}
}
test("RDDs with one-to-one dependencies") {
testCheckpointing(_.map(x => x.toString))
testCheckpointing(_.flatMap(x => 1 to x))
testCheckpointing(_.filter(_ % 2 == 0))
testCheckpointing(_.sample(false, 0.5, 0))
testCheckpointing(_.glom())
testCheckpointing(_.mapPartitions(_.map(_.toString)))
testCheckpointing(r => new MapPartitionsWithSplitRDD(r,
(i: Int, iter: Iterator[Int]) => iter.map(_.toString), false ))
testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString))
testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x))
testCheckpointing(_.pipe(Seq("cat")))
}
test("ParallelCollection") {
val parCollection = sc.makeRDD(1 to 4, 2)
val numSplits = parCollection.splits.size
parCollection.checkpoint()
assert(parCollection.dependencies === Nil)
val result = parCollection.collect()
assert(sc.checkpointFile[Int](parCollection.getCheckpointFile.get).collect() === result)
assert(parCollection.dependencies != Nil)
assert(parCollection.splits.length === numSplits)
assert(parCollection.splits.toList === parCollection.checkpointData.get.getSplits.toList)
assert(parCollection.collect() === result)
}
test("BlockRDD") {
val blockId = "id"
val blockManager = SparkEnv.get.blockManager
blockManager.putSingle(blockId, "test", StorageLevel.MEMORY_ONLY)
val blockRDD = new BlockRDD[String](sc, Array(blockId))
val numSplits = blockRDD.splits.size
blockRDD.checkpoint()
val result = blockRDD.collect()
assert(sc.checkpointFile[String](blockRDD.getCheckpointFile.get).collect() === result)
assert(blockRDD.dependencies != Nil)
assert(blockRDD.splits.length === numSplits)
assert(blockRDD.splits.toList === blockRDD.checkpointData.get.getSplits.toList)
assert(blockRDD.collect() === result)
}
test("ShuffledRDD") {
testCheckpointing(rdd => {
// Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD
new ShuffledRDD(rdd.map(x => (x % 2, 1)), partitioner)
})
}
test("UnionRDD") {
def otherRDD = sc.makeRDD(1 to 10, 1)
// Test whether the size of UnionRDDSplits reduce in size after parent RDD is checkpointed.
// Current implementation of UnionRDD has transient reference to parent RDDs,
// so only the splits will reduce in serialized size, not the RDD.
testCheckpointing(_.union(otherRDD), false, true)
testParentCheckpointing(_.union(otherRDD), false, true)
}
test("CartesianRDD") {
def otherRDD = sc.makeRDD(1 to 10, 1)
testCheckpointing(new CartesianRDD(sc, _, otherRDD))
// Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
// Current implementation of CoalescedRDDSplit has transient reference to parent RDD,
// so only the RDD will reduce in serialized size, not the splits.
testParentCheckpointing(new CartesianRDD(sc, _, otherRDD), true, false)
// Test that the CartesianRDD updates parent splits (CartesianRDD.s1/s2) after
// the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits.
// Note that this test is very specific to the current implementation of CartesianRDD.
val ones = sc.makeRDD(1 to 100, 10).map(x => x)
ones.checkpoint // checkpoint that MappedRDD
val cartesian = new CartesianRDD(sc, ones, ones)
val splitBeforeCheckpoint =
serializeDeserialize(cartesian.splits.head.asInstanceOf[CartesianSplit])
cartesian.count() // do the checkpointing
val splitAfterCheckpoint =
serializeDeserialize(cartesian.splits.head.asInstanceOf[CartesianSplit])
assert(
(splitAfterCheckpoint.s1 != splitBeforeCheckpoint.s1) &&
(splitAfterCheckpoint.s2 != splitBeforeCheckpoint.s2),
"CartesianRDD.parents not updated after parent RDD checkpointed"
)
}
test("CoalescedRDD") {
testCheckpointing(new CoalescedRDD(_, 2))
// Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
// Current implementation of CoalescedRDDSplit has transient reference to parent RDD,
// so only the RDD will reduce in serialized size, not the splits.
testParentCheckpointing(new CoalescedRDD(_, 2), true, false)
// Test that the CoalescedRDDSplit updates parent splits (CoalescedRDDSplit.parents) after
// the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits.
// Note that this test is very specific to the current implementation of CoalescedRDDSplits
val ones = sc.makeRDD(1 to 100, 10).map(x => x)
ones.checkpoint // checkpoint that MappedRDD
val coalesced = new CoalescedRDD(ones, 2)
val splitBeforeCheckpoint =
serializeDeserialize(coalesced.splits.head.asInstanceOf[CoalescedRDDSplit])
coalesced.count() // do the checkpointing
val splitAfterCheckpoint =
serializeDeserialize(coalesced.splits.head.asInstanceOf[CoalescedRDDSplit])
assert(
splitAfterCheckpoint.parents.head != splitBeforeCheckpoint.parents.head,
"CoalescedRDDSplit.parents not updated after parent RDD checkpointed"
)
}
test("CoGroupedRDD") {
val longLineageRDD1 = generateLongLineageRDDForCoGroupedRDD()
testCheckpointing(rdd => {
CheckpointSuite.cogroup(longLineageRDD1, rdd.map(x => (x % 2, 1)), partitioner)
}, false, true)
val longLineageRDD2 = generateLongLineageRDDForCoGroupedRDD()
testParentCheckpointing(rdd => {
CheckpointSuite.cogroup(
longLineageRDD2, sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)), partitioner)
}, false, true)
}
test("ZippedRDD") {
testCheckpointing(
rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
// Test whether size of ZippedRDD reduce in size after parent RDD is checkpointed
// Current implementation of ZippedRDDSplit has transient references to parent RDDs,
// so only the RDD will reduce in serialized size, not the splits.
testParentCheckpointing(
rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
}
/**
* Test checkpointing of the final RDD generated by the given operation. By default,
* this method tests whether the size of serialized RDD has reduced after checkpointing or not.
* It can also test whether the size of serialized RDD splits has reduced after checkpointing or
* not, but this is not done by default as usually the splits do not refer to any RDD and
* therefore never store the lineage.
*/
def testCheckpointing[U: ClassManifest](
op: (RDD[Int]) => RDD[U],
testRDDSize: Boolean = true,
testRDDSplitSize: Boolean = false
) {
// Generate the final RDD using given RDD operation
val baseRDD = generateLongLineageRDD
val operatedRDD = op(baseRDD)
val parentRDD = operatedRDD.dependencies.headOption.orNull
val rddType = operatedRDD.getClass.getSimpleName
val numSplits = operatedRDD.splits.length
// Find serialized sizes before and after the checkpoint
val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
operatedRDD.checkpoint()
val result = operatedRDD.collect()
val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
// Test whether the checkpoint file has been created
assert(sc.checkpointFile[U](operatedRDD.getCheckpointFile.get).collect() === result)
// Test whether dependencies have been changed from its earlier parent RDD
assert(operatedRDD.dependencies.head.rdd != parentRDD)
// Test whether the splits have been changed to the new Hadoop splits
assert(operatedRDD.splits.toList === operatedRDD.checkpointData.get.getSplits.toList)
// Test whether the number of splits is same as before
assert(operatedRDD.splits.length === numSplits)
// Test whether the data in the checkpointed RDD is same as original
assert(operatedRDD.collect() === result)
// Test whether serialized size of the RDD has reduced. If the RDD
// does not have any dependency to another RDD (e.g., ParallelCollection,
// ShuffleRDD with ShuffleDependency), it may not reduce in size after checkpointing.
if (testRDDSize) {
logInfo("Size of " + rddType +
"[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]")
assert(
rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
"Size of " + rddType + " did not reduce after checkpointing " +
"[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
)
}
// Test whether serialized size of the splits has reduced. If the splits
// do not have any non-transient reference to another RDD or another RDD's splits, it
// does not refer to a lineage and therefore may not reduce in size after checkpointing.
// However, if the original splits before checkpointing do refer to a parent RDD, the splits
// must be forgotten after checkpointing (to remove all reference to parent RDDs) and
// replaced with the HadoopSplits of the checkpointed RDD.
if (testRDDSplitSize) {
logInfo("Size of " + rddType + " splits "
+ "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]")
assert(
splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
"Size of " + rddType + " splits did not reduce after checkpointing " +
"[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
)
}
}
/**
* Test whether checkpointing of the parent of the generated RDD also
* truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent
* RDDs splits. So even if the parent RDD is checkpointed and its splits changed,
* this RDD will remember the splits and therefore potentially the whole lineage.
*/
def testParentCheckpointing[U: ClassManifest](
op: (RDD[Int]) => RDD[U],
testRDDSize: Boolean,
testRDDSplitSize: Boolean
) {
// Generate the final RDD using given RDD operation
val baseRDD = generateLongLineageRDD
val operatedRDD = op(baseRDD)
val parentRDD = operatedRDD.dependencies.head.rdd
val rddType = operatedRDD.getClass.getSimpleName
val parentRDDType = parentRDD.getClass.getSimpleName
// Find serialized sizes before and after the checkpoint
val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
parentRDD.checkpoint() // checkpoint the parent RDD, not the generated one
val result = operatedRDD.collect()
val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
// Test whether the data in the checkpointed RDD is same as original
assert(operatedRDD.collect() === result)
// Test whether serialized size of the RDD has reduced because of its parent being
// checkpointed. If this RDD or its parent RDD do not have any dependency
// to another RDD (e.g., ParallelCollection, ShuffleRDD with ShuffleDependency), it may
// not reduce in size after checkpointing.
if (testRDDSize) {
assert(
rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
"Size of " + rddType + " did not reduce after parent checkpointing parent " + parentRDDType +
"[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
)
}
// Test whether serialized size of the splits has reduced because of its parent being
// checkpointed. If the splits do not have any non-transient reference to another RDD
// or another RDD's splits, it does not refer to a lineage and therefore may not reduce
// in size after checkpointing. However, if the splits do refer to the *splits* of a parent
// RDD, then these splits must update reference to the parent RDD splits as the parent RDD's
// splits must have changed after checkpointing.
if (testRDDSplitSize) {
assert(
splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
"Size of " + rddType + " splits did not reduce after checkpointing parent " + parentRDDType +
"[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
)
}
}
/**
* Generate an RDD with a long lineage of one-to-one dependencies.
*/
def generateLongLineageRDD(): RDD[Int] = {
var rdd = sc.makeRDD(1 to 100, 4)
for (i <- 1 to 50) {
rdd = rdd.map(x => x + 1)
}
rdd
}
/**
* Generate an RDD with a long lineage specifically for CoGroupedRDD.
* A CoGroupedRDD can have a long lineage only one of its parents have a long lineage
* and narrow dependency with this RDD. This method generate such an RDD by a sequence
* of cogroups and mapValues which creates a long lineage of narrow dependencies.
*/
def generateLongLineageRDDForCoGroupedRDD() = {
val add = (x: (Seq[Int], Seq[Int])) => (x._1 ++ x._2).reduce(_ + _)
def ones: RDD[(Int, Int)] = sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _)
var cogrouped: RDD[(Int, (Seq[Int], Seq[Int]))] = ones.cogroup(ones)
for(i <- 1 to 10) {
cogrouped = cogrouped.mapValues(add).cogroup(ones)
}
cogrouped.mapValues(add)
}
/**
* Get serialized sizes of the RDD and its splits
*/
def getSerializedSizes(rdd: RDD[_]): (Int, Int) = {
(Utils.serialize(rdd).size, Utils.serialize(rdd.splits).size)
}
/**
* Serialize and deserialize an object. This is useful to verify the objects
* contents after deserialization (e.g., the contents of an RDD split after
* it is sent to a slave along with a task)
*/
def serializeDeserialize[T](obj: T): T = {
val bytes = Utils.serialize(obj)
Utils.deserialize[T](bytes)
}
}
object CheckpointSuite {
// This is a custom cogroup function that does not use mapValues like
// the PairRDDFunctions.cogroup()
def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = {
//println("First = " + first + ", second = " + second)
new CoGroupedRDD[K](
Seq(first.asInstanceOf[RDD[(_, _)]], second.asInstanceOf[RDD[(_, _)]]),
part
).asInstanceOf[RDD[(K, Seq[Seq[V]])]]
}
}

View file

@ -47,6 +47,8 @@ object TestObject {
val nums = sc.parallelize(Array(1, 2, 3, 4))
val answer = nums.map(_ + x).reduce(_ + _)
sc.stop()
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.master.port")
return answer
}
}

View file

@ -586,7 +586,7 @@ public class JavaAPISuite implements Serializable {
public void accumulators() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
final Accumulator<Integer> intAccum = sc.accumulator(10);
final Accumulator<Integer> intAccum = sc.intAccumulator(10);
rdd.foreach(new VoidFunction<Integer>() {
public void call(Integer x) {
intAccum.add(x);
@ -594,7 +594,7 @@ public class JavaAPISuite implements Serializable {
});
Assert.assertEquals((Integer) 25, intAccum.value());
final Accumulator<Double> doubleAccum = sc.accumulator(10.0);
final Accumulator<Double> doubleAccum = sc.doubleAccumulator(10.0);
rdd.foreach(new VoidFunction<Integer>() {
public void call(Integer x) {
doubleAccum.add((double) x);
@ -641,4 +641,31 @@ public class JavaAPISuite implements Serializable {
Assert.assertEquals(new Tuple2<String, Integer>("1", 1), s.get(0));
Assert.assertEquals(new Tuple2<String, Integer>("2", 2), s.get(1));
}
@Test
public void checkpointAndComputation() {
File tempDir = Files.createTempDir();
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
Assert.assertEquals(false, rdd.isCheckpointed());
rdd.checkpoint();
rdd.count(); // Forces the DAG to cause a checkpoint
Assert.assertEquals(true, rdd.isCheckpointed());
Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect());
}
@Test
public void checkpointAndRestore() {
File tempDir = Files.createTempDir();
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
Assert.assertEquals(false, rdd.isCheckpointed());
rdd.checkpoint();
rdd.count(); // Forces the DAG to cause a checkpoint
Assert.assertEquals(true, rdd.isCheckpointed());
Assert.assertTrue(rdd.getCheckpointFile().isPresent());
JavaRDD<Integer> recovered = sc.checkpointFile(rdd.getCheckpointFile().get());
Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect());
}
}

View file

@ -76,10 +76,23 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5)))
}
test("checkpointing") {
test("basic checkpointing") {
import java.io.File
val checkpointDir = File.createTempFile("temp", "")
checkpointDir.delete()
sc = new SparkContext("local", "test")
val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).flatMap(x => 1 to x).checkpoint()
assert(rdd.collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4))
sc.setCheckpointDir(checkpointDir.toString)
val parCollection = sc.makeRDD(1 to 4)
val flatMappedRDD = parCollection.flatMap(x => 1 to x)
flatMappedRDD.checkpoint()
assert(flatMappedRDD.dependencies.head.rdd == parCollection)
val result = flatMappedRDD.collect()
Thread.sleep(1000)
assert(flatMappedRDD.dependencies.head.rdd != parCollection)
assert(flatMappedRDD.collect() === result)
checkpointDir.deleteOnExit()
}
test("basic caching") {
@ -94,9 +107,9 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
sc = new SparkContext("local", "test")
val onlySplit = new Split { override def index: Int = 0 }
var shouldFail = true
val rdd = new RDD[Int](sc) {
override def splits: Array[Split] = Array(onlySplit)
override val dependencies = List[Dependency[_]]()
val rdd = new RDD[Int](sc, Nil) {
override def getSplits: Array[Split] = Array(onlySplit)
override val getDependencies = List[Dependency[_]]()
override def compute(split: Split, context: TaskContext): Iterator[Int] = {
if (shouldFail) {
throw new Exception("injected failure")
@ -123,8 +136,8 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
List(List(1, 2, 3, 4, 5), List(6, 7, 8, 9, 10)))
// Check that the narrow dependency is also specified correctly
assert(coalesced1.dependencies.head.getParents(0).toList === List(0, 1, 2, 3, 4))
assert(coalesced1.dependencies.head.getParents(1).toList === List(5, 6, 7, 8, 9))
assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(0).toList === List(0, 1, 2, 3, 4))
assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(1).toList === List(5, 6, 7, 8, 9))
val coalesced2 = new CoalescedRDD(data, 3)
assert(coalesced2.collect().toList === (1 to 10).toList)

View file

@ -23,9 +23,8 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter {
test("Calls executeOnCompleteCallbacks after failure") {
var completed = false
sc = new SparkContext("local", "test")
val rdd = new RDD[String](sc) {
override val splits = Array[Split](StubSplit(0))
override val dependencies = List()
val rdd = new RDD[String](sc, List()) {
override def getSplits = Array[Split](StubSplit(0))
override def compute(split: Split, context: TaskContext) = {
context.addOnCompleteCallback(() => completed = true)
sys.error("failed")

View file

@ -0,0 +1,23 @@
package spark.util
import org.scalatest.FunSuite
import java.io.ByteArrayOutputStream
import java.util.concurrent.TimeUnit._
class RateLimitedOutputStreamSuite extends FunSuite {
private def benchmark[U](f: => U): Long = {
val start = System.nanoTime
f
System.nanoTime - start
}
test("write") {
val underlying = new ByteArrayOutputStream
val data = "X" * 41000
val stream = new RateLimitedOutputStream(underlying, 10000)
val elapsedNs = benchmark { stream.write(data.getBytes("UTF-8")) }
assert(SECONDS.convert(elapsedNs, NANOSECONDS) == 4)
assert(underlying.toString("UTF-8") == data)
}
}

View file

@ -48,14 +48,16 @@
<li><a href="scala-programming-guide.html">Scala</a></li>
<li><a href="java-programming-guide.html">Java</a></li>
<li><a href="python-programming-guide.html">Python</a></li>
<li><a href="streaming-programming-guide.html">Spark Streaming</a></li>
</ul>
</li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown">API<b class="caret"></b></a>
<a href="#" class="dropdown-toggle" data-toggle="dropdown">API (Scaladoc)<b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="api/core/index.html">Scala/Java (Scaladoc)</a></li>
<li><a href="api/pyspark/index.html">Python (Epydoc)</a></li>
<li><a href="api/core/index.html">Spark Scala/Java (Scaladoc)</a></li>
<li><a href="api/pyspark/index.html">Spark Python (Epydoc)</a></li>
<li><a href="api/streaming/index.html">Spark Streaming Scala/Java (Scaladoc) </a></li>
</ul>
</li>

View file

@ -2,7 +2,7 @@ require 'fileutils'
include FileUtils
if ENV['SKIP_SCALADOC'] != '1'
projects = ["core", "examples", "repl", "bagel"]
projects = ["core", "examples", "repl", "bagel", "streaming"]
puts "Moving to project root and building scaladoc."
curr_dir = pwd
@ -11,7 +11,7 @@ if ENV['SKIP_SCALADOC'] != '1'
puts "Running sbt/sbt doc from " + pwd + "; this may take a few minutes..."
puts `sbt/sbt doc`
puts "moving back into docs dir."
puts "Moving back into docs dir."
cd("docs")
# Copy over the scaladoc from each project into the docs directory.

View file

@ -5,7 +5,8 @@ title: Spark API documentation (Scaladoc)
Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory.
- [Core](api/core/index.html)
- [Examples](api/examples/index.html)
- [Spark](api/core/index.html)
- [Spark Examples](api/examples/index.html)
- [Spark Streaming](api/streaming/index.html)
- [Bagel](api/bagel/index.html)
- [PySpark](api/pyspark/index.html)

View file

@ -197,6 +197,15 @@ Apart from these, the following properties are also available, and may be useful
poor data locality, but the default generally works well.
</td>
</tr>
<tr>
<td>spark.akka.frameSize</td>
<td>10</td>
<td>
Maximum message size to allow in "control plane" communication (for serialized tasks and task
results), in MB. Increase this if your tasks need to send back large results to the master
(e.g. using <code>collect()</code> on a large dataset).
</td>
</tr>
<tr>
<td>spark.akka.threads</td>
<td>4</td>
@ -205,6 +214,13 @@ Apart from these, the following properties are also available, and may be useful
when the master has a lot of CPU cores.
</td>
</tr>
<tr>
<td>spark.akka.timeout</td>
<td>20</td>
<td>
Communication timeout between Spark nodes.
</td>
</tr>
<tr>
<td>spark.master.host</td>
<td>(local hostname)</td>
@ -219,6 +235,17 @@ Apart from these, the following properties are also available, and may be useful
Port for the master to listen on.
</td>
</tr>
<tr>
<td>spark.cleaner.delay</td>
<td>(disable)</td>
<td>
Duration (minutes) of how long Spark will remember any metadata (stages generated, tasks generated, etc.).
Periodic cleanups will ensure that metadata older than this duration will be forgetten. This is
useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming
applications). Note that any RDD that persists in memory for more than this duration will be cleared as well.
</td>
</tr>
</table>
# Configuring Logging

View file

@ -96,7 +96,9 @@ permissions on your private key file, you can run `launch` with the
`spark-ec2` to attach a persistent EBS volume to each node for
storing the persistent HDFS.
- Finally, if you get errors while running your jobs, look at the slave's logs
for that job using the Mesos web UI (`http://<master-hostname>:8080`).
for that job inside of the Mesos work directory (/mnt/mesos-work). You can
also view the status of the cluster using the Mesos web UI
(`http://<master-hostname>:8080`).
# Configuration

View file

@ -58,13 +58,15 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`).
* [Quick Start](quick-start.html): a quick introduction to the Spark API; start here!
* [Spark Programming Guide](scala-programming-guide.html): an overview of Spark concepts, and details on the Scala API
* [Streaming Programming Guide](streaming-programming-guide.html): an API preview of Spark Streaming
* [Java Programming Guide](java-programming-guide.html): using Spark from Java
* [Python Programming Guide](python-programming-guide.html): using Spark from Python
**API Docs:**
* [Java/Scala (Scaladoc)](api/core/index.html)
* [Python (Epydoc)](api/pyspark/index.html)
* [Spark Java/Scala (Scaladoc)](api/core/index.html)
* [Spark Python (Epydoc)](api/pyspark/index.html)
* [Spark Streaming Java/Scala (Scaladoc)](api/streaming/index.html)
**Deployment guides:**

View file

@ -16,7 +16,6 @@ There are a few key differences between the Python and Scala APIs:
* Python is dynamically typed, so RDDs can hold objects of different types.
* PySpark does not currently support the following Spark features:
- Accumulators
- Special functions on RDDs of doubles, such as `mean` and `stdev`
- `lookup`
- `persist` at storage levels other than `MEMORY_ONLY`

View file

@ -51,11 +51,11 @@ Finally, the following configuration options can be passed to the master and wor
</tr>
<tr>
<td><code>-c CORES</code>, <code>--cores CORES</code></td>
<td>Number of CPU cores to use (default: all available); only on worker</td>
<td>Total CPU cores to allow Spark jobs to use on the machine (default: all available); only on worker</td>
</tr>
<tr>
<td><code>-m MEM</code>, <code>--memory MEM</code></td>
<td>Amount of memory to use, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
<td>Total amount of memory to allow Spark jobs to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
</tr>
<tr>
<td><code>-d DIR</code>, <code>--work-dir DIR</code></td>
@ -66,9 +66,20 @@ Finally, the following configuration options can be passed to the master and wor
# Cluster Launch Scripts
To launch a Spark standalone cluster with the deploy scripts, you need to set up two files, `conf/spark-env.sh` and `conf/slaves`. The `conf/spark-env.sh` file lets you specify global settings for the master and slave instances, such as memory, or port numbers to bind to, while `conf/slaves` is a list of slave nodes. The system requires that all the slave machines have the same configuration files, so *copy these files to each machine*.
To launch a Spark standalone cluster with the deploy scripts, you need to create a file called `conf/slaves` in your Spark directory, which should contain the hostnames of all the machines where you would like to start Spark workers, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing, you can just put `localhost` in this file.
In `conf/spark-env.sh`, you can set the following parameters, in addition to the [standard Spark configuration settings](configuration.html):
Once you've set up this fine, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
- `bin/start-master.sh` - Starts a master instance on the machine the script is executed on.
- `bin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
- `bin/start-all.sh` - Starts both a master and a number of slaves as described above.
- `bin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
- `bin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
- `bin/stop-all.sh` - Stops both the master and the slaves as described above.
Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine.
You can optionally configure the cluster further by setting environment variables in `conf/spark-env.sh`. Create this file by starting with the `conf/spark-env.sh.template`, and _copy it to all your worker machines_ for the settings to take effect. The following settings are available:
<table class="table">
<tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr>
@ -88,36 +99,24 @@ In `conf/spark-env.sh`, you can set the following parameters, in addition to the
<td><code>SPARK_WORKER_PORT</code></td>
<td>Start the Spark worker on a specific port (default: random)</td>
</tr>
<tr>
<td><code>SPARK_WORKER_DIR</code></td>
<td>Directory to run jobs in, which will include both logs and scratch space (default: SPARK_HOME/work)</td>
</tr>
<tr>
<td><code>SPARK_WORKER_CORES</code></td>
<td>Number of cores to use (default: all available cores)</td>
<td>Total number of cores to allow Spark jobs to use on the machine (default: all available cores)</td>
</tr>
<tr>
<td><code>SPARK_WORKER_MEMORY</code></td>
<td>How much memory to use, e.g. 1000M, 2G (default: total memory minus 1 GB)</td>
<td>Total amount of memory to allow Spark jobs to use on the machine, e.g. 1000M, 2G (default: total memory minus 1 GB); note that each job's <i>individual</i> memory is configured using <code>SPARK_MEM</code></td>
</tr>
<tr>
<td><code>SPARK_WORKER_WEBUI_PORT</code></td>
<td>Port for the worker web UI (default: 8081)</td>
</tr>
<tr>
<td><code>SPARK_WORKER_DIR</code></td>
<td>Directory to run jobs in, which will include both logs and scratch space (default: SPARK_HOME/work)</td>
</tr>
</table>
In `conf/slaves`, include a list of all machines where you would like to start a Spark worker, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing purposes, you can have a single `localhost` entry in the slaves file.
Once you've set up these configuration files, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
- `bin/start-master.sh` - Starts a master instance on the machine the script is executed on.
- `bin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
- `bin/start-all.sh` - Starts both a master and a number of slaves as described above.
- `bin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
- `bin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
- `bin/stop-all.sh` - Stops both the master and the slaves as described above.
Note that the scripts must be executed on the machine you want to run the Spark master on, not your local machine.
# Connecting a Job to the Cluster

View file

@ -0,0 +1,313 @@
---
layout: global
title: Spark Streaming Programming Guide
---
* This will become a table of contents (this text will be scraped).
{:toc}
# Overview
A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collection of elements) representing a continuous stream of data. DStreams can created from live incoming data (such as data from a socket, Kafka, etc.) or it can be generated by transformation of existing DStreams using parallel operators like map, reduce, and window. The basic processing model is as follows:
(i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides the data into batches. Each batch of data is treated as a RDD, that is a immutable and parallel collection of data. These input data RDDs are automatically persisted in memory (serialized by default) and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively referred to as an InputDStream.
(ii) Data received by InputDStreams are processed processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures.
This guide shows some how to start programming with DStreams.
# Initializing Spark Streaming
The first thing a Spark Streaming program must do is create a `StreamingContext` object, which tells Spark how to access a cluster. A `StreamingContext` can be created by using
{% highlight scala %}
new StreamingContext(master, jobName, batchDuration)
{% endhighlight %}
The `master` parameter is either the [Mesos master URL](running-on-mesos.html) (for running on a cluster)or the special "local" string (for local mode) that is used to create a Spark Context. For more information about this please refer to the [Spark programming guide](scala-programming-guide.html). The `jobName` is the name of the streaming job, which is the same as the jobName used in SparkContext. It is used to identify this job in the Mesos web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Starting with something conservative like 5 seconds maybe a good start. See [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion.
This constructor creates a SparkContext object using the given `master` and `jobName` parameters. However, if you already have a SparkContext or you need to create a custom SparkContext by specifying list of JARs, then a StreamingContext can be created from the existing SparkContext, by using
{% highlight scala %}
new StreamingContext(sparkContext, batchDuration)
{% endhighlight %}
# Attaching Input Sources - InputDStreams
The StreamingContext is used to creating InputDStreams from input sources:
{% highlight scala %}
// Assuming ssc is the StreamingContext
ssc.networkStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port
ssc.textFileStream(directory) // Creates a stream by monitoring and processing new files in a HDFS directory
{% endhighlight %}
A complete list of input sources is available in the [StreamingContext API documentation](api/streaming/index.html#spark.streaming.StreamingContext). Data received from these sources can be processed using DStream operations, which are explained next.
# DStream Operations
Once an input DStream has been created, you can transform it using _DStream operators_. Most of these operators return new DStreams which you can further transform. Eventually, you'll need to call an _output operator_, which forces evaluation of the DStream by writing data out to an external source.
## Transformations
DStreams support many of the transformations available on normal Spark RDD's:
<table class="table">
<tr><th style="width:25%">Transformation</th><th>Meaning</th></tr>
<tr>
<td> <b>map</b>(<i>func</i>) </td>
<td> Returns a new DStream formed by passing each element of the source through a function <i>func</i>. </td>
</tr>
<tr>
<td> <b>filter</b>(<i>func</i>) </td>
<td> Returns a new stream formed by selecting those elements of the source on which <i>func</i> returns true. </td>
</tr>
<tr>
<td> <b>flatMap</b>(<i>func</i>) </td>
<td> Similar to map, but each input item can be mapped to 0 or more output items (so <i>func</i> should return a Seq rather than a single item). </td>
</tr>
<tr>
<td> <b>mapPartitions</b>(<i>func</i>) </td>
<td> Similar to map, but runs separately on each partition (block) of the DStream, so <i>func</i> must be of type
Iterator[T] => Iterator[U] when running on an DStream of type T. </td>
</tr>
<tr>
<td> <b>union</b>(<i>otherStream</i>) </td>
<td> Return a new stream that contains the union of the elements in the source stream and the argument. </td>
</tr>
<tr>
<td> <b>groupByKey</b>([<i>numTasks</i>]) </td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, Seq[V]) pairs. <br />
<b>Note:</b> By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.
</td>
</tr>
<tr>
<td> <b>reduceByKey</b>(<i>func</i>, [<i>numTasks</i>]) </td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, V) pairs where the values for each key are aggregated using the given reduce function. Like in <code>groupByKey</code>, the number of reduce tasks is configurable through an optional second argument. </td>
</tr>
<tr>
<td> <b>join</b>(<i>otherStream</i>, [<i>numTasks</i>]) </td>
<td> When called on streams of type (K, V) and (K, W), returns a stream of (K, (V, W)) pairs with all pairs of elements for each key. </td>
</tr>
<tr>
<td> <b>cogroup</b>(<i>otherStream</i>, [<i>numTasks</i>]) </td>
<td> When called on DStream of type (K, V) and (K, W), returns a DStream of (K, Seq[V], Seq[W]) tuples.</td>
</tr>
<tr>
<td> <b>reduce</b>(<i>func</i>) </td>
<td> Returns a new DStream of single-element RDDs by aggregating the elements of the stream using a function func (which takes two arguments and returns one). The function should be associative so that it can be computed correctly in parallel. </td>
</tr>
<tr>
<td> <b>transform</b>(<i>func</i>) </td>
<td> Returns a new DStream by applying func (a RDD-to-RDD function) to every RDD of the stream. This can be used to do arbitrary RDD operations on the DStream. </td>
</tr>
</table>
Spark Streaming features windowed computations, which allow you to report statistics over a sliding window of data. All window functions take a <i>windowDuration</i>, which represents the width of the window and a <i>slideTime</i>, which represents the frequency during which the window is calculated.
<table class="table">
<tr><th style="width:25%">Transformation</th><th>Meaning</th></tr>
<tr>
<td> <b>window</b>(<i>windowDuration</i>, </i>slideTime</i>) </td>
<td> Return a new stream which is computed based on windowed batches of the source stream. <i>windowDuration</i> is the width of the window and <i>slideTime</i> is the frequency during which the window is calculated. Both times must be multiples of the batch interval.
</td>
</tr>
<tr>
<td> <b>countByWindow</b>(<i>windowDuration</i>, </i>slideTime</i>) </td>
<td> Return a sliding count of elements in the stream. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td>
</tr>
<tr>
<td> <b>reduceByWindow</b>(<i>func</i>, <i>windowDuration</i>, </i>slideDuration</i>) </td>
<td> Return a new single-element stream, created by aggregating elements in the stream over a sliding interval using <i>func</i>. The function should be associative so that it can be computed correctly in parallel. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td>
</tr>
<tr>
<td> <b>groupByKeyAndWindow</b>(windowDuration, slideDuration, [<i>numTasks</i>])
</td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, Seq[V]) pairs over a sliding window. <br />
<b>Note:</b> By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td>
</tr>
<tr>
<td> <b>reduceByKeyAndWindow</b>(<i>func</i>, [<i>numTasks</i>]) </td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, V) pairs where the values for each key are aggregated using the given reduce function over batches within a sliding window. Like in <code>groupByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument.
<i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td>
</tr>
<tr>
<td> <b>countByKeyAndWindow</b>([<i>numTasks</i>]) </td>
<td> When called on a stream of (K, V) pairs, returns a stream of (K, Int) pairs where the values for each key are the count within a sliding window. Like in <code>countByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument.
<i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td>
</tr>
</table>
A complete list of DStream operations is available in the API documentation of [DStream](api/streaming/index.html#spark.streaming.DStream) and [PairDStreamFunctions](api/streaming/index.html#spark.streaming.PairDStreamFunctions).
## Output Operations
When an output operator is called, it triggers the computation of a stream. Currently the following output operators are defined:
<table class="table">
<tr><th style="width:25%">Operator</th><th>Meaning</th></tr>
<tr>
<td> <b>foreach</b>(<i>func</i>) </td>
<td> The fundamental output operator. Applies a function, <i>func</i>, to each RDD generated from the stream. This function should have side effects, such as printing output, saving the RDD to external files, or writing it over the network to an external system. </td>
</tr>
<tr>
<td> <b>print</b>() </td>
<td> Prints first ten elements of every batch of data in a DStream on the driver. </td>
</tr>
<tr>
<td> <b>saveAsObjectFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
<td> Save this DStream's contents as a <code>SequenceFile</code> of serialized objects. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>.
</td>
</tr>
<tr>
<td> <b>saveAsTextFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
<td> Save this DStream's contents as a text files. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
</tr>
<tr>
<td> <b>saveAsHadoopFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
<td> Save this DStream's contents as a Hadoop file. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
</tr>
</table>
## DStream Persistence
Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple DStream operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`.
Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More information on different persistence levels can be found in [Spark Programming Guide](scala-programming-guide.html#rdd-persistence).
# Starting the Streaming computation
All the above DStream operations are completely lazy, that is, the operations will start executing only after the context is started by using
{% highlight scala %}
ssc.start()
{% endhighlight %}
Conversely, the computation can be stopped by using
{% highlight scala %}
ssc.stop()
{% endhighlight %}
# Example - NetworkWordCount.scala
A good example to start off is the spark.streaming.examples.NetworkWordCount. This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in <Spark repo>/streaming/src/main/scala/spark/streaming/examples/WordCountNetwork.scala.
{% highlight scala %}
import spark.streaming.{Seconds, StreamingContext}
import spark.streaming.StreamingContext._
...
// Create the context and set up a network input stream to receive from a host:port
val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1))
val lines = ssc.networkTextStream(args(1), args(2).toInt)
// Split the lines into words, count them, and print some of the counts on the master
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
// Start the computation
ssc.start()
{% endhighlight %}
To run this example on your local machine, you need to first run a Netcat server by using
{% highlight bash %}
$ nc -lk 9999
{% endhighlight %}
Then, in a different terminal, you can start NetworkWordCount by using
{% highlight bash %}
$ ./run spark.streaming.examples.NetworkWordCount local[2] localhost 9999
{% endhighlight %}
This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen.
<table>
<td>
{% highlight bash %}
# TERMINAL 1
# RUNNING NETCAT
$ nc -lk 9999
hello world
...
{% endhighlight %}
</td>
<td>
{% highlight bash %}
# TERMINAL 2: RUNNING NetworkWordCount
...
2012-12-31 18:47:10,446 INFO SparkContext: Job finished: run at ThreadPoolExecutor.java:886, took 0.038817 s
-------------------------------------------
Time: 1357008430000 ms
-------------------------------------------
(hello,1)
(world,1)
2012-12-31 18:47:10,447 INFO JobManager: Total delay: 0.44700 s for job 8 (execution: 0.44000 s)
...
{% endhighlight %}
</td>
</table>
# Performance Tuning
Getting the best performance of a Spark Streaming application on a cluster requires a bit of tuning. This section explains a number of the parameters and configurations that can tuned to improve the performance of you application. At a high level, you need to consider two things:
<ol>
<li>Reducing the processing time of each batch of data by efficiently using cluster resources.</li>
<li>Setting the right batch size such that the data processing can keep up with the data ingestion.</li>
</ol>
## Reducing the Processing Time of each Batch
There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones.
### Level of Parallelism
Cluster resources maybe underutilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`spark.PairDStreamFunctions`](api/streaming/index.html#spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default.
### Data Serialization
The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it.
* Serialization of RDD data in Spark: Please refer to the detailed discussion on data serialization in the [Tuning Guide](tuning.html). However, note that unlike Spark, by default RDDs are persisted as serialized byte arrays to minimize pauses related to GC.
* Serialization of input data: To ingest external data into Spark, data received as bytes (say, from the network) needs to deserialized from bytes and re-serialized into Spark's serialization format. Hence, the deserialization overhead of input data may be a bottleneck.
### Task Launching Overheads
If the number of tasks launched per second is high (say, 50 or more per second), then the overhead of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes:
* Task Serialization: Using Kryo serialization for serializing tasks can reduced the task sizes, and therefore reduce the time taken to send them to the slaves.
* Execution mode: Running Spark in Standalone mode or coarse-grained Mesos mode leads to better task launch times than the fine-grained Mesos mode. Please refer to the [Running on Mesos guide](running-on-mesos.html) for more details.
These changes may reduce batch processing time by 100s of milliseconds, thus allowing sub-second batch size to be viable.
## Setting the Right Batch Size
For a Spark Streaming application running on a cluster to be stable, the processing of the data streams must keep up with the rate of ingestion of the data streams. Depending on the type of computation, the batch size used may have significant impact on the rate of ingestion that can be sustained by the Spark Streaming application on a fixed cluster resources. For example, let us consider the earlier WordCountNetwork example. For a particular data rate, the system may be able to keep up with reporting word counts every 2 seconds (i.e., batch size of 2 seconds), but not every 500 milliseconds.
A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size).
## 24/7 Operation
By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.delay` to the number of minutes you want any metadata to persist. For example, setting `spark.cleaner.delay` to 10 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created.
This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so.
## Memory Tuning
Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, we highlight a few customizations that are strongly recommended to minimize GC related pauses in Spark Streaming applications and achieving more consistent batch processing times.
* <b>Default persistence level of DStreams</b>: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, [StorageLevel.MEMORY_ONLY_SER](api/core/index.html#spark.storage.StorageLevel$) for DStream compared to [StorageLevel.MEMORY_ONLY](api/core/index.html#spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs a higher serialization overheads, it significantly reduces GC pauses.
* <b>Concurrent garbage collector</b>: Using the concurrent mark-and-sweep GC further minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times.
# Master Fault-tolerance (Alpha)
TODO
* Checkpointing of DStream graph
* Recovery from master faults
* Current state and future directions

View file

@ -19,6 +19,11 @@
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
</dependency>
<dependency>
<groupId>org.twitter4j</groupId>
<artifactId>twitter4j-stream</artifactId>
<version>3.0.3</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
@ -57,6 +62,12 @@
<version>${project.version}</version>
<classifier>hadoop1</classifier>
</dependency>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-streaming</artifactId>
<version>${project.version}</version>
<classifier>hadoop1</classifier>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
@ -90,6 +101,12 @@
<version>${project.version}</version>
<classifier>hadoop2</classifier>
</dependency>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-streaming</artifactId>
<version>${project.version}</version>
<classifier>hadoop2</classifier>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>

View file

@ -7,6 +7,7 @@ import cern.jet.math._
import cern.colt.matrix._
import cern.colt.matrix.linalg._
import spark._
import scala.Option
object SparkALS {
// Parameters set through command line arguments
@ -42,7 +43,7 @@ object SparkALS {
return sqrt(sumSqs / (M * U))
}
def updateMovie(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D],
def update(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D],
R: DoubleMatrix2D) : DoubleMatrix1D =
{
val U = us.size
@ -68,50 +69,30 @@ object SparkALS {
return solved2D.viewColumn(0)
}
def updateUser(j: Int, u: DoubleMatrix1D, ms: Array[DoubleMatrix1D],
R: DoubleMatrix2D) : DoubleMatrix1D =
{
val M = ms.size
val F = ms(0).size
val XtX = factory2D.make(F, F)
val Xty = factory1D.make(F)
// For each movie that the user rated
for (i <- 0 until M) {
val m = ms(i)
// Add m * m^t to XtX
blas.dger(1, m, m, XtX)
// Add m * rating to Xty
blas.daxpy(R.get(i, j), m, Xty)
}
// Add regularization coefs to diagonal terms
for (d <- 0 until F) {
XtX.set(d, d, XtX.get(d, d) + LAMBDA * M)
}
// Solve it with Cholesky
val ch = new CholeskyDecomposition(XtX)
val Xty2D = factory2D.make(Xty.toArray, F)
val solved2D = ch.solve(Xty2D)
return solved2D.viewColumn(0)
}
def main(args: Array[String]) {
var host = ""
var slices = 0
args match {
case Array(m, u, f, iters, slices_, host_) => {
M = m.toInt
U = u.toInt
F = f.toInt
ITERATIONS = iters.toInt
slices = slices_.toInt
host = host_
(0 to 5).map(i => {
i match {
case a if a < args.length => Some(args(a))
case _ => None
}
}).toArray match {
case Array(host_, m, u, f, iters, slices_) => {
host = host_ getOrElse "local"
M = (m getOrElse "100").toInt
U = (u getOrElse "500").toInt
F = (f getOrElse "10").toInt
ITERATIONS = (iters getOrElse "5").toInt
slices = (slices_ getOrElse "2").toInt
}
case _ => {
System.err.println("Usage: SparkALS <M> <U> <F> <iters> <slices> <master>")
System.err.println("Usage: SparkALS [<master> <M> <U> <F> <iters> <slices>]")
System.exit(1)
}
}
printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS);
printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS)
val spark = new SparkContext(host, "SparkALS")
val R = generateR()
@ -127,11 +108,11 @@ object SparkALS {
for (iter <- 1 to ITERATIONS) {
println("Iteration " + iter + ":")
ms = spark.parallelize(0 until M, slices)
.map(i => updateMovie(i, msc.value(i), usc.value, Rc.value))
.map(i => update(i, msc.value(i), usc.value, Rc.value))
.toArray
msc = spark.broadcast(ms) // Re-broadcast ms because it was updated
us = spark.parallelize(0 until U, slices)
.map(i => updateUser(i, usc.value(i), msc.value, Rc.value))
.map(i => update(i, usc.value(i), msc.value, algebra.transpose(Rc.value)))
.toArray
usc = spark.broadcast(us) // Re-broadcast us because it was updated
println("RMSE = " + rmse(R, ms, us))

View file

@ -0,0 +1,43 @@
package spark.streaming.examples
import spark.util.IntParam
import spark.storage.StorageLevel
import spark.streaming._
/**
* Produces a count of events received from Flume.
*
* This should be used in conjunction with an AvroSink in Flume. It will start
* an Avro server on at the request host:port address and listen for requests.
* Your Flume AvroSink should be pointed to this address.
*
* Usage: FlumeEventCount <master> <host> <port>
*
* <master> is a Spark master URL
* <host> is the host the Flume receiver will be started on - a receiver
* creates a server and listens for flume events.
* <port> is the port the Flume receiver will listen on.
*/
object FlumeEventCount {
def main(args: Array[String]) {
if (args.length != 3) {
System.err.println(
"Usage: FlumeEventCount <master> <host> <port>")
System.exit(1)
}
val Array(master, host, IntParam(port)) = args
val batchInterval = Milliseconds(2000)
// Create the context and set the batch size
val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval)
// Create a flume stream
val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY)
// Print out the count of events received from this server in each batch
stream.count().map(cnt => "Received " + cnt + " flume events." ).print()
ssc.start()
}
}

View file

@ -0,0 +1,36 @@
package spark.streaming.examples
import spark.streaming.{Seconds, StreamingContext}
import spark.streaming.StreamingContext._
/**
* Counts words in new text files created in the given directory
* Usage: HdfsWordCount <master> <directory>
* <master> is the Spark master URL.
* <directory> is the directory that Spark Streaming will use to find and read new text files.
*
* To run this on your local machine on directory `localdir`, run this example
* `$ ./run spark.streaming.examples.HdfsWordCount local[2] localdir`
* Then create a text file in `localdir` and the words in the file will get counted.
*/
object HdfsWordCount {
def main(args: Array[String]) {
if (args.length < 2) {
System.err.println("Usage: HdfsWordCount <master> <directory>")
System.exit(1)
}
// Create the context
val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2))
// Create the FileInputDStream on the directory and use the
// stream to count words in new files created
val lines = ssc.textFileStream(args(1))
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
ssc.start()
}
}

View file

@ -0,0 +1,50 @@
package spark.streaming.examples;
import spark.api.java.function.Function;
import spark.streaming.*;
import spark.streaming.api.java.*;
import spark.streaming.dstream.SparkFlumeEvent;
/**
* Produces a count of events received from Flume.
*
* This should be used in conjunction with an AvroSink in Flume. It will start
* an Avro server on at the request host:port address and listen for requests.
* Your Flume AvroSink should be pointed to this address.
*
* Usage: JavaFlumeEventCount <master> <host> <port>
*
* <master> is a Spark master URL
* <host> is the host the Flume receiver will be started on - a receiver
* creates a server and listens for flume events.
* <port> is the port the Flume receiver will listen on.
*/
public class JavaFlumeEventCount {
public static void main(String[] args) {
if (args.length != 3) {
System.err.println("Usage: JavaFlumeEventCount <master> <host> <port>");
System.exit(1);
}
String master = args[0];
String host = args[1];
int port = Integer.parseInt(args[2]);
Duration batchInterval = new Duration(2000);
JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval);
JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port);
flumeStream.count();
flumeStream.count().map(new Function<Long, String>() {
@Override
public String call(Long in) {
return "Received " + in + " flume events.";
}
}).print();
sc.start();
}
}

View file

@ -0,0 +1,62 @@
package spark.streaming.examples;
import com.google.common.collect.Lists;
import scala.Tuple2;
import spark.api.java.function.FlatMapFunction;
import spark.api.java.function.Function2;
import spark.api.java.function.PairFunction;
import spark.streaming.Duration;
import spark.streaming.api.java.JavaDStream;
import spark.streaming.api.java.JavaPairDStream;
import spark.streaming.api.java.JavaStreamingContext;
/**
* Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
* Usage: NetworkWordCount <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.
*
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
* `$ ./run spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999`
*/
public class JavaNetworkWordCount {
public static void main(String[] args) {
if (args.length < 2) {
System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
"In local mode, <master> should be 'local[n]' with n > 1");
System.exit(1);
}
// Create the context with a 1 second batch size
JavaStreamingContext ssc = new JavaStreamingContext(
args[0], "NetworkWordCount", new Duration(1000));
// Create a NetworkInputDStream on target ip:port and count the
// words in input stream of \n delimited test (eg. generated by 'nc')
JavaDStream<String> lines = ssc.networkTextStream(args[1], Integer.parseInt(args[2]));
JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {
return Lists.newArrayList(x.split(" "));
}
});
JavaPairDStream<String, Integer> wordCounts = words.map(
new PairFunction<String, String, Integer>() {
@Override
public Tuple2<String, Integer> call(String s) throws Exception {
return new Tuple2<String, Integer>(s, 1);
}
}).reduceByKey(new Function2<Integer, Integer, Integer>() {
@Override
public Integer call(Integer i1, Integer i2) throws Exception {
return i1 + i2;
}
});
wordCounts.print();
ssc.start();
}
}

View file

@ -0,0 +1,62 @@
package spark.streaming.examples;
import com.google.common.collect.Lists;
import scala.Tuple2;
import spark.api.java.JavaRDD;
import spark.api.java.function.Function2;
import spark.api.java.function.PairFunction;
import spark.streaming.Duration;
import spark.streaming.api.java.JavaDStream;
import spark.streaming.api.java.JavaPairDStream;
import spark.streaming.api.java.JavaStreamingContext;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
public class JavaQueueStream {
public static void main(String[] args) throws InterruptedException {
if (args.length < 1) {
System.err.println("Usage: JavaQueueStream <master>");
System.exit(1);
}
// Create the context
JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000));
// Create the queue through which RDDs can be pushed to
// a QueueInputDStream
Queue<JavaRDD<Integer>> rddQueue = new LinkedList<JavaRDD<Integer>>();
// Create and push some RDDs into the queue
List<Integer> list = Lists.newArrayList();
for (int i = 0; i < 1000; i++) {
list.add(i);
}
for (int i = 0; i < 30; i++) {
rddQueue.add(ssc.sc().parallelize(list));
}
// Create the QueueInputDStream and use it do some processing
JavaDStream<Integer> inputStream = ssc.queueStream(rddQueue);
JavaPairDStream<Integer, Integer> mappedStream = inputStream.map(
new PairFunction<Integer, Integer, Integer>() {
@Override
public Tuple2<Integer, Integer> call(Integer i) throws Exception {
return new Tuple2<Integer, Integer>(i % 10, 1);
}
});
JavaPairDStream<Integer, Integer> reducedStream = mappedStream.reduceByKey(
new Function2<Integer, Integer, Integer>() {
@Override
public Integer call(Integer i1, Integer i2) throws Exception {
return i1 + i2;
}
});
reducedStream.print();
ssc.start();
}
}

View file

@ -0,0 +1,69 @@
package spark.streaming.examples
import java.util.Properties
import kafka.message.Message
import kafka.producer.SyncProducerConfig
import kafka.producer._
import spark.SparkContext
import spark.streaming._
import spark.streaming.StreamingContext._
import spark.storage.StorageLevel
import spark.streaming.util.RawTextHelper._
object KafkaWordCount {
def main(args: Array[String]) {
if (args.length < 6) {
System.err.println("Usage: KafkaWordCount <master> <hostname> <port> <group> <topics> <numThreads>")
System.exit(1)
}
val Array(master, hostname, port, group, topics, numThreads) = args
val sc = new SparkContext(master, "KafkaWordCount")
val ssc = new StreamingContext(sc, Seconds(2))
ssc.checkpoint("checkpoint")
val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap
val lines = ssc.kafkaStream[String](hostname, port.toInt, group, topicpMap)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1l)).reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)
wordCounts.print()
ssc.start()
}
}
// Produces some random words between 1 and 100.
object KafkaWordCountProducer {
def main(args: Array[String]) {
if (args.length < 3) {
System.err.println("Usage: KafkaWordCountProducer <hostname> <port> <topic> <messagesPerSec> <wordsPerMessage>")
System.exit(1)
}
val Array(hostname, port, topic, messagesPerSec, wordsPerMessage) = args
// Zookeper connection properties
val props = new Properties()
props.put("zk.connect", hostname + ":" + port)
props.put("serializer.class", "kafka.serializer.StringEncoder")
val config = new ProducerConfig(props)
val producer = new Producer[String, String](config)
// Send some messages
while(true) {
val messages = (1 to messagesPerSec.toInt).map { messageNum =>
(1 to wordsPerMessage.toInt).map(x => scala.util.Random.nextInt(10).toString).mkString(" ")
}.toArray
println(messages.mkString(","))
val data = new ProducerData[String, String](topic, messages)
producer.send(data)
Thread.sleep(100)
}
}
}

View file

@ -0,0 +1,36 @@
package spark.streaming.examples
import spark.streaming.{Seconds, StreamingContext}
import spark.streaming.StreamingContext._
/**
* Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
* Usage: NetworkWordCount <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.
*
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
* `$ ./run spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
*/
object NetworkWordCount {
def main(args: Array[String]) {
if (args.length < 2) {
System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
"In local mode, <master> should be 'local[n]' with n > 1")
System.exit(1)
}
// Create the context with a 1 second batch size
val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1))
// Create a NetworkInputDStream on target ip:port and count the
// words in input stream of \n delimited test (eg. generated by 'nc')
val lines = ssc.networkTextStream(args(1), args(2).toInt)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
ssc.start()
}
}

View file

@ -0,0 +1,39 @@
package spark.streaming.examples
import spark.RDD
import spark.streaming.{Seconds, StreamingContext}
import spark.streaming.StreamingContext._
import scala.collection.mutable.SynchronizedQueue
object QueueStream {
def main(args: Array[String]) {
if (args.length < 1) {
System.err.println("Usage: QueueStream <master>")
System.exit(1)
}
// Create the context
val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1))
// Create the queue through which RDDs can be pushed to
// a QueueInputDStream
val rddQueue = new SynchronizedQueue[RDD[Int]]()
// Create the QueueInputDStream and use it do some processing
val inputStream = ssc.queueStream(rddQueue)
val mappedStream = inputStream.map(x => (x % 10, 1))
val reducedStream = mappedStream.reduceByKey(_ + _)
reducedStream.print()
ssc.start()
// Create and push some RDDs into
for (i <- 1 to 30) {
rddQueue += ssc.sc.makeRDD(1 to 1000, 10)
Thread.sleep(1000)
}
ssc.stop()
System.exit(0)
}
}

View file

@ -0,0 +1,46 @@
package spark.streaming.examples
import spark.util.IntParam
import spark.storage.StorageLevel
import spark.streaming._
import spark.streaming.util.RawTextHelper
/**
* Receives text from multiple rawNetworkStreams and counts how many '\n' delimited
* lines have the word 'the' in them. This is useful for benchmarking purposes. This
* will only work with spark.streaming.util.RawTextSender running on all worker nodes
* and with Spark using Kryo serialization (set Java property "spark.serializer" to
* "spark.KryoSerializer").
* Usage: RawNetworkGrep <master> <numStreams> <host> <port> <batchMillis>
* <master> is the Spark master URL
* <numStream> is the number rawNetworkStreams, which should be same as number
* of work nodes in the cluster
* <host> is "localhost".
* <port> is the port on which RawTextSender is running in the worker nodes.
* <batchMillise> is the Spark Streaming batch duration in milliseconds.
*/
object RawNetworkGrep {
def main(args: Array[String]) {
if (args.length != 5) {
System.err.println("Usage: RawNetworkGrep <master> <numStreams> <host> <port> <batchMillis>")
System.exit(1)
}
val Array(master, IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args
// Create the context
val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis))
// Warm up the JVMs on master and slave for JIT compilation to kick in
RawTextHelper.warmUp(ssc.sc)
val rawStreams = (1 to numStreams).map(_ =>
ssc.rawNetworkStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
val union = ssc.union(rawStreams)
union.filter(_.contains("the")).count().foreach(r =>
println("Grep count: " + r.collect().mkString))
ssc.start()
}
}

View file

@ -0,0 +1,85 @@
package spark.streaming.examples.clickstream
import java.net.{InetAddress,ServerSocket,Socket,SocketException}
import java.io.{InputStreamReader, BufferedReader, PrintWriter}
import util.Random
/** Represents a page view on a website with associated dimension data.*/
class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int) {
override def toString() : String = {
"%s\t%s\t%s\t%s\n".format(url, status, zipCode, userID)
}
}
object PageView {
def fromString(in : String) : PageView = {
val parts = in.split("\t")
new PageView(parts(0), parts(1).toInt, parts(2).toInt, parts(3).toInt)
}
}
/** Generates streaming events to simulate page views on a website.
*
* This should be used in tandem with PageViewStream.scala. Example:
* $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10
* $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
* */
object PageViewGenerator {
val pages = Map("http://foo.com/" -> .7,
"http://foo.com/news" -> 0.2,
"http://foo.com/contact" -> .1)
val httpStatus = Map(200 -> .95,
404 -> .05)
val userZipCode = Map(94709 -> .5,
94117 -> .5)
val userID = Map((1 to 100).map(_ -> .01):_*)
def pickFromDistribution[T](inputMap : Map[T, Double]) : T = {
val rand = new Random().nextDouble()
var total = 0.0
for ((item, prob) <- inputMap) {
total = total + prob
if (total > rand) {
return item
}
}
return inputMap.take(1).head._1 // Shouldn't get here if probabilities add up to 1.0
}
def getNextClickEvent() : String = {
val id = pickFromDistribution(userID)
val page = pickFromDistribution(pages)
val status = pickFromDistribution(httpStatus)
val zipCode = pickFromDistribution(userZipCode)
new PageView(page, status, zipCode, id).toString()
}
def main(args : Array[String]) {
if (args.length != 2) {
System.err.println("Usage: PageViewGenerator <port> <viewsPerSecond>")
System.exit(1)
}
val port = args(0).toInt
val viewsPerSecond = args(1).toFloat
val sleepDelayMs = (1000.0 / viewsPerSecond).toInt
val listener = new ServerSocket(port)
println("Listening on port: " + port)
while (true) {
val socket = listener.accept()
new Thread() {
override def run = {
println("Got client connected from: " + socket.getInetAddress)
val out = new PrintWriter(socket.getOutputStream(), true)
while (true) {
Thread.sleep(sleepDelayMs)
out.write(getNextClickEvent())
out.flush()
}
socket.close()
}
}.start()
}
}
}

View file

@ -0,0 +1,84 @@
package spark.streaming.examples.clickstream
import spark.streaming.{Seconds, StreamingContext}
import spark.streaming.StreamingContext._
import spark.SparkContext._
/** Analyses a streaming dataset of web page views. This class demonstrates several types of
* operators available in Spark streaming.
*
* This should be used in tandem with PageViewStream.scala. Example:
* $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10
* $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
* */
object PageViewStream {
def main(args: Array[String]) {
if (args.length != 3) {
System.err.println("Usage: PageViewStream <metric> <host> <port>")
System.err.println("<metric> must be one of pageCounts, slidingPageCounts," +
" errorRatePerZipCode, activeUserCount, popularUsersSeen")
System.exit(1)
}
val metric = args(0)
val host = args(1)
val port = args(2).toInt
// Create the context
val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1))
// Create a NetworkInputDStream on target host:port and convert each line to a PageView
val pageViews = ssc.networkTextStream(host, port)
.flatMap(_.split("\n"))
.map(PageView.fromString(_))
// Return a count of views per URL seen in each batch
val pageCounts = pageViews.map(view => ((view.url, 1))).countByKey()
// Return a sliding window of page views per URL in the last ten seconds
val slidingPageCounts = pageViews.map(view => ((view.url, 1)))
.window(Seconds(10), Seconds(2))
.countByKey()
// Return the rate of error pages (a non 200 status) in each zip code over the last 30 seconds
val statusesPerZipCode = pageViews.window(Seconds(30), Seconds(2))
.map(view => ((view.zipCode, view.status)))
.groupByKey()
val errorRatePerZipCode = statusesPerZipCode.map{
case(zip, statuses) =>
val normalCount = statuses.filter(_ == 200).size
val errorCount = statuses.size - normalCount
val errorRatio = errorCount.toFloat / statuses.size
if (errorRatio > 0.05) {"%s: **%s**".format(zip, errorRatio)}
else {"%s: %s".format(zip, errorRatio)}
}
// Return the number unique users in last 15 seconds
val activeUserCount = pageViews.window(Seconds(15), Seconds(2))
.map(view => (view.userID, 1))
.groupByKey()
.count()
.map("Unique active users: " + _)
// An external dataset we want to join to this stream
val userList = ssc.sc.parallelize(
Map(1 -> "Patrick Wendell", 2->"Reynold Xin", 3->"Matei Zaharia").toSeq)
metric match {
case "pageCounts" => pageCounts.print()
case "slidingPageCounts" => slidingPageCounts.print()
case "errorRatePerZipCode" => errorRatePerZipCode.print()
case "activeUserCount" => activeUserCount.print()
case "popularUsersSeen" =>
// Look for users in our existing dataset and print it out if we have a match
pageViews.map(view => (view.userID, 1))
.foreach((rdd, time) => rdd.join(userList)
.map(_._2._2)
.take(10)
.foreach(u => println("Saw user %s at time %s".format(u, time))))
case _ => println("Invalid metric entered: " + metric)
}
ssc.start()
}
}

View file

@ -0,0 +1,60 @@
package spark.streaming.examples.twitter
import spark.streaming.StreamingContext._
import spark.streaming.{Seconds, StreamingContext}
import spark.SparkContext._
import spark.storage.StorageLevel
/**
* Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter
* stream. The stream is instantiated with credentials and optionally filters supplied by the
* command line arguments.
*/
object TwitterBasic {
def main(args: Array[String]) {
if (args.length < 3) {
System.err.println("Usage: TwitterBasic <master> <twitter_username> <twitter_password>" +
" [filter1] [filter2] ... [filter n]")
System.exit(1)
}
val Array(master, username, password) = args.slice(0, 3)
val filters = args.slice(3, args.length)
val ssc = new StreamingContext(master, "TwitterBasic", Seconds(2))
val stream = new TwitterInputDStream(ssc, username, password, filters,
StorageLevel.MEMORY_ONLY_SER)
ssc.registerInputStream(stream)
val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#")))
val topCounts60 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(60))
.map{case (topic, count) => (count, topic)}
.transform(_.sortByKey(false))
val topCounts10 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(10))
.map{case (topic, count) => (count, topic)}
.transform(_.sortByKey(false))
// Print popular hashtags
topCounts60.foreach(rdd => {
if (rdd.count() != 0) {
val topList = rdd.take(5)
println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count()))
topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
}
})
topCounts10.foreach(rdd => {
if (rdd.count() != 0) {
val topList = rdd.take(5)
println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count()))
topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
}
})
ssc.start()
}
}

View file

@ -0,0 +1,71 @@
package spark.streaming.examples.twitter
import spark._
import spark.streaming._
import dstream.{NetworkReceiver, NetworkInputDStream}
import storage.StorageLevel
import twitter4j._
import twitter4j.auth.BasicAuthorization
import collection.JavaConversions._
/* A stream of Twitter statuses, potentially filtered by one or more keywords.
*
* @constructor create a new Twitter stream using the supplied username and password to authenticate.
* An optional set of string filters can be used to restrict the set of tweets. The Twitter API is
* such that this may return a sampled subset of all tweets during each interval.
*/
class TwitterInputDStream(
@transient ssc_ : StreamingContext,
username: String,
password: String,
filters: Seq[String],
storageLevel: StorageLevel
) extends NetworkInputDStream[Status](ssc_) {
override def createReceiver(): NetworkReceiver[Status] = {
new TwitterReceiver(username, password, filters, storageLevel)
}
}
class TwitterReceiver(
username: String,
password: String,
filters: Seq[String],
storageLevel: StorageLevel
) extends NetworkReceiver[Status] {
var twitterStream: TwitterStream = _
lazy val blockGenerator = new BlockGenerator(storageLevel)
protected override def onStart() {
blockGenerator.start()
twitterStream = new TwitterStreamFactory()
.getInstance(new BasicAuthorization(username, password))
twitterStream.addListener(new StatusListener {
def onStatus(status: Status) = {
blockGenerator += status
}
// Unimplemented
def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {}
def onTrackLimitationNotice(i: Int) {}
def onScrubGeo(l: Long, l1: Long) {}
def onStallWarning(stallWarning: StallWarning) {}
def onException(e: Exception) {}
})
val query: FilterQuery = new FilterQuery
if (filters.size > 0) {
query.track(filters.toArray)
twitterStream.filter(query)
} else {
twitterStream.sample()
}
logInfo("Twitter receiver started")
}
protected override def onStop() {
blockGenerator.stop()
twitterStream.shutdown()
logInfo("Twitter receiver stopped")
}
}

12
pom.xml
View file

@ -41,6 +41,7 @@
<module>core</module>
<module>bagel</module>
<module>examples</module>
<module>streaming</module>
<module>repl</module>
<module>repl-bin</module>
</modules>
@ -104,6 +105,17 @@
<enabled>false</enabled>
</snapshots>
</repository>
<repository>
<id>twitter4j-repo</id>
<name>Twitter4J Repository</name>
<url>http://twitter4j.org/maven2/</url>
<releases>
<enabled>true</enabled>
</releases>
<snapshots>
<enabled>false</enabled>
</snapshots>
</repository>
</repositories>
<pluginRepositories>
<pluginRepository>

View file

@ -17,16 +17,18 @@ object SparkBuild extends Build {
//val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1"
//val HADOOP_MAJOR_VERSION = "2"
lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel)
lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming)
lazy val core = Project("core", file("core"), settings = coreSettings)
lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core)
lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) dependsOn (streaming)
lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core)
lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming)
lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core)
lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core)
// A configuration to set an alternative publishLocalConfiguration
lazy val MavenCompile = config("m2r") extend(Compile)
lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
@ -114,7 +116,8 @@ object SparkBuild extends Build {
"Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/",
"JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/",
"Spray Repository" at "http://repo.spray.cc/",
"Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/"
"Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/",
"Twitter4J Repository" at "http://twitter4j.org/maven2/"
),
libraryDependencies ++= Seq(
@ -150,11 +153,22 @@ object SparkBuild extends Build {
)
def examplesSettings = sharedSettings ++ Seq(
name := "spark-examples"
name := "spark-examples",
libraryDependencies ++= Seq(
"org.twitter4j" % "twitter4j-stream" % "3.0.3"
)
)
def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel")
def streamingSettings = sharedSettings ++ Seq(
name := "spark-streaming",
libraryDependencies ++= Seq(
"org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile",
"com.github.sgroschupf" % "zkclient" % "0.1"
)
) ++ assemblySettings ++ extraAssemblySettings
def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq(
mergeStrategy in assembly := {
case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard

View file

@ -16,4 +16,4 @@ target: docs/
private: no
exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.serializers
pyspark.java_gateway pyspark.examples pyspark.shell
pyspark.java_gateway pyspark.examples pyspark.shell pyspark.test

71
python/examples/als.py Executable file
View file

@ -0,0 +1,71 @@
"""
This example requires numpy (http://www.numpy.org/)
"""
from os.path import realpath
import sys
import numpy as np
from numpy.random import rand
from numpy import matrix
from pyspark import SparkContext
LAMBDA = 0.01 # regularization
np.random.seed(42)
def rmse(R, ms, us):
diff = R - ms * us.T
return np.sqrt(np.sum(np.power(diff, 2)) / M * U)
def update(i, vec, mat, ratings):
uu = mat.shape[0]
ff = mat.shape[1]
XtX = matrix(np.zeros((ff, ff)))
Xty = np.zeros((ff, 1))
for j in range(uu):
v = mat[j, :]
XtX += v.T * v
Xty += v.T * ratings[i, j]
XtX += np.eye(ff, ff) * LAMBDA * uu
return np.linalg.solve(XtX, Xty)
if __name__ == "__main__":
if len(sys.argv) < 2:
print >> sys.stderr, \
"Usage: PythonALS <master> <M> <U> <F> <iters> <slices>"
exit(-1)
sc = SparkContext(sys.argv[1], "PythonALS", pyFiles=[realpath(__file__)])
M = int(sys.argv[2]) if len(sys.argv) > 2 else 100
U = int(sys.argv[3]) if len(sys.argv) > 3 else 500
F = int(sys.argv[4]) if len(sys.argv) > 4 else 10
ITERATIONS = int(sys.argv[5]) if len(sys.argv) > 5 else 5
slices = int(sys.argv[6]) if len(sys.argv) > 6 else 2
print "Running ALS with M=%d, U=%d, F=%d, iters=%d, slices=%d\n" % \
(M, U, F, ITERATIONS, slices)
R = matrix(rand(M, F)) * matrix(rand(U, F).T)
ms = matrix(rand(M ,F))
us = matrix(rand(U, F))
Rb = sc.broadcast(R)
msb = sc.broadcast(ms)
usb = sc.broadcast(us)
for i in range(ITERATIONS):
ms = sc.parallelize(range(M), slices) \
.map(lambda x: update(x, msb.value[x, :], usb.value, Rb.value)) \
.collect()
ms = matrix(np.array(ms)[:, :, 0]) # collect() returns a list, so array ends up being
# a 3-d array, we take the first 2 dims for the matrix
msb = sc.broadcast(ms)
us = sc.parallelize(range(U), slices) \
.map(lambda x: update(x, usb.value[x, :], msb.value, Rb.value.T)) \
.collect()
us = matrix(np.array(us)[:, :, 0])
usb = sc.broadcast(us)
error = rmse(R, ms, us)
print "Iteration %d:" % i
print "\nRMSE: %5.4f\n" % error

View file

@ -7,6 +7,10 @@ Public classes:
Main entry point for Spark functionality.
- L{RDD<pyspark.rdd.RDD>}
A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
- L{Broadcast<pyspark.broadcast.Broadcast>}
A broadcast variable that gets reused across tasks.
- L{Accumulator<pyspark.accumulators.Accumulator>}
An "add-only" shared variable that tasks can only add values to.
"""
import sys
import os

View file

@ -0,0 +1,187 @@
"""
>>> from pyspark.context import SparkContext
>>> sc = SparkContext('local', 'test')
>>> a = sc.accumulator(1)
>>> a.value
1
>>> a.value = 2
>>> a.value
2
>>> a += 5
>>> a.value
7
>>> sc.accumulator(1.0).value
1.0
>>> sc.accumulator(1j).value
1j
>>> rdd = sc.parallelize([1,2,3])
>>> def f(x):
... global a
... a += x
>>> rdd.foreach(f)
>>> a.value
13
>>> class VectorAccumulatorParam(object):
... def zero(self, value):
... return [0.0] * len(value)
... def addInPlace(self, val1, val2):
... for i in xrange(len(val1)):
... val1[i] += val2[i]
... return val1
>>> va = sc.accumulator([1.0, 2.0, 3.0], VectorAccumulatorParam())
>>> va.value
[1.0, 2.0, 3.0]
>>> def g(x):
... global va
... va += [x] * 3
>>> rdd.foreach(g)
>>> va.value
[7.0, 8.0, 9.0]
>>> rdd.map(lambda x: a.value).collect() # doctest: +IGNORE_EXCEPTION_DETAIL
Traceback (most recent call last):
...
Py4JJavaError:...
>>> def h(x):
... global a
... a.value = 7
>>> rdd.foreach(h) # doctest: +IGNORE_EXCEPTION_DETAIL
Traceback (most recent call last):
...
Py4JJavaError:...
>>> sc.accumulator([1.0, 2.0, 3.0]) # doctest: +IGNORE_EXCEPTION_DETAIL
Traceback (most recent call last):
...
Exception:...
"""
import struct
import SocketServer
import threading
from pyspark.cloudpickle import CloudPickler
from pyspark.serializers import read_int, read_with_length, load_pickle
# Holds accumulators registered on the current machine, keyed by ID. This is then used to send
# the local accumulator updates back to the driver program at the end of a task.
_accumulatorRegistry = {}
def _deserialize_accumulator(aid, zero_value, accum_param):
from pyspark.accumulators import _accumulatorRegistry
accum = Accumulator(aid, zero_value, accum_param)
accum._deserialized = True
_accumulatorRegistry[aid] = accum
return accum
class Accumulator(object):
"""
A shared variable that can be accumulated, i.e., has a commutative and associative "add"
operation. Worker tasks on a Spark cluster can add values to an Accumulator with the C{+=}
operator, but only the driver program is allowed to access its value, using C{value}.
Updates from the workers get propagated automatically to the driver program.
While C{SparkContext} supports accumulators for primitive data types like C{int} and
C{float}, users can also define accumulators for custom types by providing a custom
C{AccumulatorParam} object with a C{zero} and C{addInPlace} method. Refer to the doctest
of this module for an example.
"""
def __init__(self, aid, value, accum_param):
"""Create a new Accumulator with a given initial value and AccumulatorParam object"""
from pyspark.accumulators import _accumulatorRegistry
self.aid = aid
self.accum_param = accum_param
self._value = value
self._deserialized = False
_accumulatorRegistry[aid] = self
def __reduce__(self):
"""Custom serialization; saves the zero value from our AccumulatorParam"""
param = self.accum_param
return (_deserialize_accumulator, (self.aid, param.zero(self._value), param))
@property
def value(self):
"""Get the accumulator's value; only usable in driver program"""
if self._deserialized:
raise Exception("Accumulator.value cannot be accessed inside tasks")
return self._value
@value.setter
def value(self, value):
"""Sets the accumulator's value; only usable in driver program"""
if self._deserialized:
raise Exception("Accumulator.value cannot be accessed inside tasks")
self._value = value
def __iadd__(self, term):
"""The += operator; adds a term to this accumulator's value"""
self._value = self.accum_param.addInPlace(self._value, term)
return self
def __str__(self):
return str(self._value)
def __repr__(self):
return "Accumulator<id=%i, value=%s>" % (self.aid, self._value)
class AddingAccumulatorParam(object):
"""
An AccumulatorParam that uses the + operators to add values. Designed for simple types
such as integers, floats, and lists. Requires the zero value for the underlying type
as a parameter.
"""
def __init__(self, zero_value):
self.zero_value = zero_value
def zero(self, value):
return self.zero_value
def addInPlace(self, value1, value2):
value1 += value2
return value1
# Singleton accumulator params for some standard types
INT_ACCUMULATOR_PARAM = AddingAccumulatorParam(0)
FLOAT_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0)
COMPLEX_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0j)
class _UpdateRequestHandler(SocketServer.StreamRequestHandler):
def handle(self):
from pyspark.accumulators import _accumulatorRegistry
num_updates = read_int(self.rfile)
for _ in range(num_updates):
(aid, update) = load_pickle(read_with_length(self.rfile))
_accumulatorRegistry[aid] += update
# Write a byte in acknowledgement
self.wfile.write(struct.pack("!b", 1))
def _start_update_server():
"""Start a TCP server to receive accumulator updates in a daemon thread, and returns it"""
server = SocketServer.TCPServer(("localhost", 0), _UpdateRequestHandler)
thread = threading.Thread(target=server.serve_forever)
thread.daemon = True
thread.start()
return server
def _test():
import doctest
doctest.testmod()
if __name__ == "__main__":
_test()

View file

@ -2,6 +2,8 @@ import os
import atexit
from tempfile import NamedTemporaryFile
from pyspark import accumulators
from pyspark.accumulators import Accumulator
from pyspark.broadcast import Broadcast
from pyspark.java_gateway import launch_gateway
from pyspark.serializers import dump_pickle, write_with_length, batched
@ -22,6 +24,7 @@ class SparkContext(object):
_readRDDFromPickleFile = jvm.PythonRDD.readRDDFromPickleFile
_writeIteratorToPickleFile = jvm.PythonRDD.writeIteratorToPickleFile
_takePartition = jvm.PythonRDD.takePartition
_next_accum_id = 0
def __init__(self, master, jobName, sparkHome=None, pyFiles=None,
environment=None, batchSize=1024):
@ -52,6 +55,14 @@ class SparkContext(object):
self._jsc = self.jvm.JavaSparkContext(master, jobName, sparkHome,
empty_string_array)
# Create a single Accumulator in Java that we'll send all our updates through;
# they will be passed back to us through a TCP server
self._accumulatorServer = accumulators._start_update_server()
(host, port) = self._accumulatorServer.server_address
self._javaAccumulator = self._jsc.accumulator(
self.jvm.java.util.ArrayList(),
self.jvm.PythonAccumulatorParam(host, port))
self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python')
# Broadcast's __reduce__ method stores Broadcast instances here.
# This allows other code to determine which Broadcast instances have
@ -74,6 +85,8 @@ class SparkContext(object):
def __del__(self):
if self._jsc:
self._jsc.stop()
if self._accumulatorServer:
self._accumulatorServer.shutdown()
def stop(self):
"""
@ -110,6 +123,10 @@ class SparkContext(object):
jrdd = self._jsc.textFile(name, minSplits)
return RDD(jrdd, self)
def _checkpointFile(self, name):
jrdd = self._jsc.checkpointFile(name)
return RDD(jrdd, self)
def union(self, rdds):
"""
Build the union of a list of RDDs.
@ -129,6 +146,31 @@ class SparkContext(object):
return Broadcast(jbroadcast.id(), value, jbroadcast,
self._pickled_broadcast_vars)
def accumulator(self, value, accum_param=None):
"""
Create an C{Accumulator} with the given initial value, using a given
AccumulatorParam helper object to define how to add values of the data
type if provided. Default AccumulatorParams are used for integers and
floating-point numbers if you do not provide one. For other types, the
AccumulatorParam must implement two methods:
- C{zero(value)}: provide a "zero value" for the type, compatible in
dimensions with the provided C{value} (e.g., a zero vector).
- C{addInPlace(val1, val2)}: add two values of the accumulator's data
type, returning a new value; for efficiency, can also update C{val1}
in place and return it.
"""
if accum_param == None:
if isinstance(value, int):
accum_param = accumulators.INT_ACCUMULATOR_PARAM
elif isinstance(value, float):
accum_param = accumulators.FLOAT_ACCUMULATOR_PARAM
elif isinstance(value, complex):
accum_param = accumulators.COMPLEX_ACCUMULATOR_PARAM
else:
raise Exception("No default accumulator param for type %s" % type(value))
SparkContext._next_accum_id += 1
return Accumulator(SparkContext._next_accum_id - 1, value, accum_param)
def addFile(self, path):
"""
Add a file to be downloaded into the working directory of this Spark
@ -157,3 +199,15 @@ class SparkContext(object):
filename = path.split("/")[-1]
os.environ["PYTHONPATH"] = \
"%s:%s" % (filename, os.environ["PYTHONPATH"])
def setCheckpointDir(self, dirName, useExisting=False):
"""
Set the directory under which RDDs are going to be checkpointed. The
directory must be a HDFS path if running on a cluster.
If the directory does not exist, it will be created. If the directory
exists and C{useExisting} is set to true, then the exisiting directory
will be used. Otherwise an exception will be thrown to prevent
accidental overriding of checkpoint files in the existing directory.
"""
self._jsc.sc().setCheckpointDir(dirName, useExisting)

View file

@ -32,7 +32,9 @@ class RDD(object):
def __init__(self, jrdd, ctx):
self._jrdd = jrdd
self.is_cached = False
self.is_checkpointed = False
self.ctx = ctx
self._partitionFunc = None
@property
def context(self):
@ -49,6 +51,34 @@ class RDD(object):
self._jrdd.cache()
return self
def checkpoint(self):
"""
Mark this RDD for checkpointing. It will be saved to a file inside the
checkpoint directory set with L{SparkContext.setCheckpointDir()} and
all references to its parent RDDs will be removed. This function must
be called before any job has been executed on this RDD. It is strongly
recommended that this RDD is persisted in memory, otherwise saving it
on a file will require recomputation.
"""
self.is_checkpointed = True
self._jrdd.rdd().checkpoint()
def isCheckpointed(self):
"""
Return whether this RDD has been checkpointed or not
"""
return self._jrdd.rdd().isCheckpointed()
def getCheckpointFile(self):
"""
Gets the name of the file to which this RDD was checkpointed
"""
checkpointFile = self._jrdd.rdd().getCheckpointFile()
if checkpointFile.isDefined():
return checkpointFile.get()
else:
return None
# TODO persist(self, storageLevel)
def map(self, f, preservesPartitioning=False):
@ -497,7 +527,7 @@ class RDD(object):
return python_right_outer_join(self, other, numSplits)
# TODO: add option to control map-side combining
def partitionBy(self, numSplits, hashFunc=hash):
def partitionBy(self, numSplits, partitionFunc=hash):
"""
Return a copy of the RDD partitioned using the specified partitioner.
@ -514,17 +544,21 @@ class RDD(object):
def add_shuffle_key(split, iterator):
buckets = defaultdict(list)
for (k, v) in iterator:
buckets[hashFunc(k) % numSplits].append((k, v))
buckets[partitionFunc(k) % numSplits].append((k, v))
for (split, items) in buckets.iteritems():
yield str(split)
yield dump_pickle(Batch(items))
keyed = PipelinedRDD(self, add_shuffle_key)
keyed._bypass_serializer = True
pairRDD = self.ctx.jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
partitioner = self.ctx.jvm.spark.api.python.PythonPartitioner(numSplits)
jrdd = pairRDD.partitionBy(partitioner)
jrdd = jrdd.map(self.ctx.jvm.ExtractValue())
return RDD(jrdd, self.ctx)
partitioner = self.ctx.jvm.PythonPartitioner(numSplits,
id(partitionFunc))
jrdd = pairRDD.partitionBy(partitioner).values()
rdd = RDD(jrdd, self.ctx)
# This is required so that id(partitionFunc) remains unique, even if
# partitionFunc is a lambda:
rdd._partitionFunc = partitionFunc
return rdd
# TODO: add control over map-side aggregation
def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
@ -662,7 +696,7 @@ class PipelinedRDD(RDD):
20
"""
def __init__(self, prev, func, preservesPartitioning=False):
if isinstance(prev, PipelinedRDD) and not prev.is_cached:
if isinstance(prev, PipelinedRDD) and prev._is_pipelinable():
prev_func = prev.func
def pipeline_func(split, iterator):
return func(split, prev_func(split, iterator))
@ -675,6 +709,7 @@ class PipelinedRDD(RDD):
self.preservesPartitioning = preservesPartitioning
self._prev_jrdd = prev._jrdd
self.is_cached = False
self.is_checkpointed = False
self.ctx = prev.ctx
self.prev = prev
self._jrdd_val = None
@ -703,10 +738,13 @@ class PipelinedRDD(RDD):
env = MapConverter().convert(env, self.ctx.gateway._gateway_client)
python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(),
pipe_command, env, self.preservesPartitioning, self.ctx.pythonExec,
broadcast_vars, class_manifest)
broadcast_vars, self.ctx._javaAccumulator, class_manifest)
self._jrdd_val = python_rdd.asJavaRDD()
return self._jrdd_val
def _is_pipelinable(self):
return not (self.is_cached or self.is_checkpointed)
def _test():
import doctest

View file

@ -52,8 +52,13 @@ def read_int(stream):
raise EOFError
return struct.unpack("!i", length)[0]
def write_int(value, stream):
stream.write(struct.pack("!i", value))
def write_with_length(obj, stream):
stream.write(struct.pack("!i", len(obj)))
write_int(len(obj), stream)
stream.write(obj)

View file

@ -1,7 +1,7 @@
"""
An interactive shell.
This fle is designed to be launched as a PYTHONSTARTUP script.
This file is designed to be launched as a PYTHONSTARTUP script.
"""
import os
from pyspark.context import SparkContext
@ -14,4 +14,4 @@ print "Spark context avaiable as sc."
# which allows us to execute the user's PYTHONSTARTUP file:
_pythonstartup = os.environ.get('OLD_PYTHONSTARTUP')
if _pythonstartup and os.path.isfile(_pythonstartup):
execfile(_pythonstartup)
execfile(_pythonstartup)

61
python/pyspark/tests.py Normal file
View file

@ -0,0 +1,61 @@
"""
Unit tests for PySpark; additional tests are implemented as doctests in
individual modules.
"""
import os
import shutil
from tempfile import NamedTemporaryFile
import time
import unittest
from pyspark.context import SparkContext
class TestCheckpoint(unittest.TestCase):
def setUp(self):
self.sc = SparkContext('local[4]', 'TestPartitioning', batchSize=2)
self.checkpointDir = NamedTemporaryFile(delete=False)
os.unlink(self.checkpointDir.name)
self.sc.setCheckpointDir(self.checkpointDir.name)
def tearDown(self):
self.sc.stop()
# To avoid Akka rebinding to the same port, since it doesn't unbind
# immediately on shutdown
self.sc.jvm.System.clearProperty("spark.master.port")
shutil.rmtree(self.checkpointDir.name)
def test_basic_checkpointing(self):
parCollection = self.sc.parallelize([1, 2, 3, 4])
flatMappedRDD = parCollection.flatMap(lambda x: range(1, x + 1))
self.assertFalse(flatMappedRDD.isCheckpointed())
self.assertIsNone(flatMappedRDD.getCheckpointFile())
flatMappedRDD.checkpoint()
result = flatMappedRDD.collect()
time.sleep(1) # 1 second
self.assertTrue(flatMappedRDD.isCheckpointed())
self.assertEqual(flatMappedRDD.collect(), result)
self.assertEqual(self.checkpointDir.name,
os.path.dirname(flatMappedRDD.getCheckpointFile()))
def test_checkpoint_and_restore(self):
parCollection = self.sc.parallelize([1, 2, 3, 4])
flatMappedRDD = parCollection.flatMap(lambda x: [x])
self.assertFalse(flatMappedRDD.isCheckpointed())
self.assertIsNone(flatMappedRDD.getCheckpointFile())
flatMappedRDD.checkpoint()
flatMappedRDD.count() # forces a checkpoint to be computed
time.sleep(1) # 1 second
self.assertIsNotNone(flatMappedRDD.getCheckpointFile())
recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile())
self.assertEquals([1, 2, 3, 4], recovered.collect())
if __name__ == "__main__":
unittest.main()

View file

@ -5,9 +5,10 @@ import sys
from base64 import standard_b64decode
# CloudPickler needs to be imported so that depicklers are registered using the
# copy_reg module.
from pyspark.accumulators import _accumulatorRegistry
from pyspark.broadcast import Broadcast, _broadcastRegistry
from pyspark.cloudpickle import CloudPickler
from pyspark.serializers import write_with_length, read_with_length, \
from pyspark.serializers import write_with_length, read_with_length, write_int, \
read_long, read_int, dump_pickle, load_pickle, read_from_pickle_file
@ -36,6 +37,10 @@ def main():
iterator = read_from_pickle_file(sys.stdin)
for obj in func(split_index, iterator):
write_with_length(dumps(obj), old_stdout)
# Mark the beginning of the accumulators section of the output
write_int(-1, old_stdout)
for aid, accum in _accumulatorRegistry.items():
write_with_length(dump_pickle((aid, accum._value)), old_stdout)
if __name__ == '__main__':

View file

@ -11,6 +11,12 @@ FAILED=$(($?||$FAILED))
$FWDIR/pyspark -m doctest pyspark/broadcast.py
FAILED=$(($?||$FAILED))
$FWDIR/pyspark -m doctest pyspark/accumulators.py
FAILED=$(($?||$FAILED))
$FWDIR/pyspark -m unittest pyspark.tests
FAILED=$(($?||$FAILED))
if [[ $FAILED != 0 ]]; then
echo -en "\033[31m" # Red
echo "Had test failures; see logs."

View file

@ -101,6 +101,13 @@
<classifier>hadoop1</classifier>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-streaming</artifactId>
<version>${project.version}</version>
<classifier>hadoop1</classifier>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>
@ -151,6 +158,13 @@
<classifier>hadoop2</classifier>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.spark-project</groupId>
<artifactId>spark-streaming</artifactId>
<version>${project.version}</version>
<classifier>hadoop2</classifier>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-core</artifactId>

View file

@ -1,8 +1,8 @@
# Set everything to be logged to the console
# Set everything to be logged to the repl/target/unit-tests.log
log4j.rootCategory=INFO, file
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
log4j.appender.file.file=spark-tests.log
log4j.appender.file.file=repl/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n

10
run
View file

@ -63,6 +63,7 @@ CORE_DIR="$FWDIR/core"
REPL_DIR="$FWDIR/repl"
EXAMPLES_DIR="$FWDIR/examples"
BAGEL_DIR="$FWDIR/bagel"
STREAMING_DIR="$FWDIR/streaming"
PYSPARK_DIR="$FWDIR/python"
# Exit if the user hasn't compiled Spark
@ -82,14 +83,17 @@ fi
CLASSPATH+=":$CORE_DIR/src/main/resources"
CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
if [ -e "$FWDIR/lib_managed" ]; then
CLASSPATH+=":$FWDIR/lib_managed/jars/*"
CLASSPATH+=":$FWDIR/lib_managed/bundles/*"
fi
CLASSPATH+=":$REPL_DIR/lib/*"
for jar in `find "$REPL_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
CLASSPATH+=":$jar"
done
if [ -e repl-bin/target ]; then
for jar in `find "repl-bin/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
CLASSPATH+=":$jar"
done
fi
CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
CLASSPATH+=":$jar"

View file

@ -0,0 +1 @@
18876b8bc2e4cef28b6d191aa49d963f

View file

@ -0,0 +1 @@
06b27270ffa52250a2c08703b397c99127b72060

View file

@ -0,0 +1,9 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<modelVersion>4.0.0</modelVersion>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka</artifactId>
<version>0.7.2-spark</version>
<description>POM was created from install:install-file</description>
</project>

Some files were not shown because too many files have changed in this diff Show more