diff --git a/.gitignore b/.gitignore index f22248f40da8bf10f775da5cae4ef7e95560e32c..c207409e3cfe087723ec38fa75fca68404bf251c 100644 --- a/.gitignore +++ b/.gitignore @@ -31,3 +31,4 @@ project/plugins/src_managed/ logs/ log/ spark-tests.log +dependency-reduced-pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml new file mode 100644 index 0000000000000000000000000000000000000000..a8256a6e8b0ea32bc028b3404c40fa70b81363e2 --- /dev/null +++ b/bagel/pom.xml @@ -0,0 +1,106 @@ +<?xml version="1.0" encoding="UTF-8"?> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <parent> + <groupId>org.spark-project</groupId> + <artifactId>parent</artifactId> + <version>0.7.0-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <groupId>org.spark-project</groupId> + <artifactId>spark-bagel</artifactId> + <packaging>jar</packaging> + <name>Spark Project Bagel</name> + <url>http://spark-project.org/</url> + + <dependencies> + <dependency> + <groupId>org.eclipse.jetty</groupId> + <artifactId>jetty-server</artifactId> + </dependency> + + <dependency> + <groupId>org.scalatest</groupId> + <artifactId>scalatest_${scala.version}</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.scalacheck</groupId> + <artifactId>scalacheck_${scala.version}</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + <build> + <outputDirectory>target/scala-${scala.version}/classes</outputDirectory> + <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory> + <plugins> + <plugin> + <groupId>org.scalatest</groupId> + <artifactId>scalatest-maven-plugin</artifactId> + </plugin> + </plugins> + </build> + + <profiles> + <profile> + <id>hadoop1</id> + <dependencies> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-core</artifactId> + <version>${project.version}</version> + <classifier>hadoop1</classifier> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>provided</scope> + </dependency> + </dependencies> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-jar-plugin</artifactId> + <configuration> + <classifier>hadoop1</classifier> + </configuration> + </plugin> + </plugins> + </build> + </profile> + <profile> + <id>hadoop2</id> + <dependencies> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-core</artifactId> + <version>${project.version}</version> + <classifier>hadoop2</classifier> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-client</artifactId> + <scope>provided</scope> + </dependency> + </dependencies> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-jar-plugin</artifactId> + <configuration> + <classifier>hadoop2</classifier> + </configuration> + </plugin> + </plugins> + </build> + </profile> + </profiles> +</project> diff --git a/bin/start-all.sh b/bin/start-all.sh index 9bd6c5065445126d2cf2d94041b77364da2ddfe9..b9891ad2f6ea5673e0044f7db754529107aea7a6 100755 --- a/bin/start-all.sh +++ b/bin/start-all.sh @@ -11,7 +11,7 @@ bin=`cd "$bin"; pwd` . "$bin/spark-config.sh" # Start Master -"$bin"/start-master.sh --config $SPARK_CONF_DIR +"$bin"/start-master.sh # Start Workers -"$bin"/start-slaves.sh --config $SPARK_CONF_DIR \ No newline at end of file +"$bin"/start-slaves.sh diff --git a/bin/start-master.sh b/bin/start-master.sh index 6403c944a4ea35111aae839d8a13751fad80191f..a901b1c26068e47ad0eb476aacf4928b0124c0b9 100755 --- a/bin/start-master.sh +++ b/bin/start-master.sh @@ -7,4 +7,28 @@ bin=`cd "$bin"; pwd` . "$bin/spark-config.sh" -"$bin"/spark-daemon.sh start spark.deploy.master.Master \ No newline at end of file +if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then + . "${SPARK_CONF_DIR}/spark-env.sh" +fi + +if [ "$SPARK_MASTER_PORT" = "" ]; then + SPARK_MASTER_PORT=7077 +fi + +if [ "$SPARK_MASTER_IP" = "" ]; then + SPARK_MASTER_IP=`hostname` +fi + +if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then + SPARK_MASTER_WEBUI_PORT=8080 +fi + +# Set SPARK_PUBLIC_DNS so the master report the correct webUI address to the slaves +if [ "$SPARK_PUBLIC_DNS" = "" ]; then + # If we appear to be running on EC2, use the public address by default: + if [[ `hostname` == *ec2.internal ]]; then + export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname` + fi +fi + +"$bin"/spark-daemon.sh start spark.deploy.master.Master --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT diff --git a/bin/start-slave.sh b/bin/start-slave.sh new file mode 100755 index 0000000000000000000000000000000000000000..45a0cf7a6b7ac526fb0651e6a3f12f4cbfca8b51 --- /dev/null +++ b/bin/start-slave.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +# Set SPARK_PUBLIC_DNS so slaves can be linked in master web UI +if [ "$SPARK_PUBLIC_DNS" = "" ]; then + # If we appear to be running on EC2, use the public address by default: + if [[ `hostname` == *ec2.internal ]]; then + export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname` + fi +fi + +"$bin"/spark-daemon.sh start spark.deploy.worker.Worker $1 diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh index 74b70a24be976b5804cca4495fcbe23674ba3279..390247ca4aa4916e0f53b001c6cb3927df02da65 100755 --- a/bin/start-slaves.sh +++ b/bin/start-slaves.sh @@ -15,20 +15,10 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_IP" = "" ]; then - hostname=`hostname` - hostouput=`host "$hostname"` - - if [[ "$hostouput" == *"not found"* ]]; then - echo $hostouput - echo "Fail to identiy the IP for the master." - echo "Set SPARK_MASTER_IP explicitly in configuration instead." - exit 1 - fi - ip=`host "$hostname" | cut -d " " -f 4` -else - ip=$SPARK_MASTER_IP + SPARK_MASTER_IP=`hostname` fi -echo "Master IP: $ip" +echo "Master IP: $SPARK_MASTER_IP" -"$bin"/spark-daemons.sh start spark.deploy.worker.Worker spark://$ip:$SPARK_MASTER_PORT \ No newline at end of file +# Launch the slaves +exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT diff --git a/core/pom.xml b/core/pom.xml new file mode 100644 index 0000000000000000000000000000000000000000..ae52c20657b5448140d97813e8eae7286135a38f --- /dev/null +++ b/core/pom.xml @@ -0,0 +1,270 @@ +<?xml version="1.0" encoding="UTF-8"?> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <parent> + <groupId>org.spark-project</groupId> + <artifactId>parent</artifactId> + <version>0.7.0-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <groupId>org.spark-project</groupId> + <artifactId>spark-core</artifactId> + <packaging>jar</packaging> + <name>Spark Project Core</name> + <url>http://spark-project.org/</url> + + <dependencies> + <dependency> + <groupId>org.eclipse.jetty</groupId> + <artifactId>jetty-server</artifactId> + </dependency> + <dependency> + <groupId>com.google.guava</groupId> + <artifactId>guava</artifactId> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + </dependency> + <dependency> + <groupId>com.ning</groupId> + <artifactId>compress-lzf</artifactId> + </dependency> + <dependency> + <groupId>asm</groupId> + <artifactId>asm-all</artifactId> + </dependency> + <dependency> + <groupId>com.google.protobuf</groupId> + <artifactId>protobuf-java</artifactId> + </dependency> + <dependency> + <groupId>de.javakaffee</groupId> + <artifactId>kryo-serializers</artifactId> + </dependency> + <dependency> + <groupId>com.typesafe.akka</groupId> + <artifactId>akka-actor</artifactId> + </dependency> + <dependency> + <groupId>com.typesafe.akka</groupId> + <artifactId>akka-remote</artifactId> + </dependency> + <dependency> + <groupId>com.typesafe.akka</groupId> + <artifactId>akka-slf4j</artifactId> + </dependency> + <dependency> + <groupId>it.unimi.dsi</groupId> + <artifactId>fastutil</artifactId> + </dependency> + <dependency> + <groupId>colt</groupId> + <artifactId>colt</artifactId> + </dependency> + <dependency> + <groupId>cc.spray</groupId> + <artifactId>spray-can</artifactId> + </dependency> + <dependency> + <groupId>cc.spray</groupId> + <artifactId>spray-server</artifactId> + </dependency> + <dependency> + <groupId>org.tomdz.twirl</groupId> + <artifactId>twirl-api</artifactId> + </dependency> + <dependency> + <groupId>com.github.scala-incubator.io</groupId> + <artifactId>scala-io-file_${scala.version}</artifactId> + </dependency> + <dependency> + <groupId>org.apache.mesos</groupId> + <artifactId>mesos</artifactId> + </dependency> + + <dependency> + <groupId>org.scalatest</groupId> + <artifactId>scalatest_${scala.version}</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.scalacheck</groupId> + <artifactId>scalacheck_${scala.version}</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>com.novocode</groupId> + <artifactId>junit-interface</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-log4j12</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + <build> + <outputDirectory>target/scala-${scala.version}/classes</outputDirectory> + <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-antrun-plugin</artifactId> + <executions> + <execution> + <phase>test</phase> + <goals> + <goal>run</goal> + </goals> + <configuration> + <exportAntProperties>true</exportAntProperties> + <tasks> + <property name="spark.classpath" refid="maven.test.classpath"/> + <property environment="env"/> + <fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry."> + <condition> + <not> + <or> + <isset property="env.SCALA_HOME"/> + <isset property="env.SCALA_LIBRARY_PATH"/> + </or> + </not> + </condition> + </fail> + </tasks> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.scalatest</groupId> + <artifactId>scalatest-maven-plugin</artifactId> + <configuration> + <environmentVariables> + <SPARK_HOME>${basedir}/..</SPARK_HOME> + <SPARK_TESTING>1</SPARK_TESTING> + <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH> + </environmentVariables> + </configuration> + </plugin> + <plugin> + <groupId>org.tomdz.twirl</groupId> + <artifactId>twirl-maven-plugin</artifactId> + </plugin> + </plugins> + </build> + + <profiles> + <profile> + <id>hadoop1</id> + <dependencies> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>provided</scope> + </dependency> + </dependencies> + <build> + <plugins> + <plugin> + <groupId>org.codehaus.mojo</groupId> + <artifactId>build-helper-maven-plugin</artifactId> + <executions> + <execution> + <id>add-source</id> + <phase>generate-sources</phase> + <goals> + <goal>add-source</goal> + </goals> + <configuration> + <sources> + <source>src/main/scala</source> + <source>src/hadoop1/scala</source> + </sources> + </configuration> + </execution> + <execution> + <id>add-scala-test-sources</id> + <phase>generate-test-sources</phase> + <goals> + <goal>add-test-source</goal> + </goals> + <configuration> + <sources> + <source>src/test/scala</source> + </sources> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-jar-plugin</artifactId> + <configuration> + <classifier>hadoop1</classifier> + </configuration> + </plugin> + </plugins> + </build> + </profile> + <profile> + <id>hadoop2</id> + <dependencies> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-client</artifactId> + <scope>provided</scope> + </dependency> + </dependencies> + <build> + <plugins> + <plugin> + <groupId>org.codehaus.mojo</groupId> + <artifactId>build-helper-maven-plugin</artifactId> + <executions> + <execution> + <id>add-source</id> + <phase>generate-sources</phase> + <goals> + <goal>add-source</goal> + </goals> + <configuration> + <sources> + <source>src/main/scala</source> + <source>src/hadoop2/scala</source> + </sources> + </configuration> + </execution> + <execution> + <id>add-scala-test-sources</id> + <phase>generate-test-sources</phase> + <goals> + <goal>add-test-source</goal> + </goals> + <configuration> + <sources> + <source>src/test/scala</source> + </sources> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-jar-plugin</artifactId> + <configuration> + <classifier>hadoop2</classifier> + </configuration> + </plugin> + </plugins> + </build> + </profile> + </profiles> +</project> \ No newline at end of file diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala index cb54e1225761bb007699b2effeeb9087493fad5e..05428aae1f2a21de2bf1ee8b4b7d7e795b76891f 100644 --- a/core/src/main/scala/spark/CacheTracker.scala +++ b/core/src/main/scala/spark/CacheTracker.scala @@ -1,5 +1,9 @@ package spark +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet + import akka.actor._ import akka.dispatch._ import akka.pattern.ask @@ -8,10 +12,6 @@ import akka.util.Duration import akka.util.Timeout import akka.util.duration._ -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet - import spark.storage.BlockManager import spark.storage.StorageLevel import util.{TimeStampedHashSet, MetadataCleaner, TimeStampedHashMap} @@ -44,7 +44,7 @@ private[spark] class CacheTrackerActor extends Actor with Logging { 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) - + def receive = { case SlaveCacheStarted(host: String, size: Long) => slaveCapacity.put(host, size) @@ -96,14 +96,14 @@ private[spark] class CacheTrackerActor extends Actor with Logging { private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: BlockManager) extends Logging { - + // Tracker actor on the master, or remote reference to it on workers val ip: String = System.getProperty("spark.master.host", "localhost") val port: Int = System.getProperty("spark.master.port", "7077").toInt val actorName: String = "CacheTracker" val timeout = 10.seconds - + var trackerActor: ActorRef = if (isMaster) { val actor = actorSystem.actorOf(Props[CacheTrackerActor], name = actorName) logInfo("Registered CacheTrackerActor actor") @@ -140,7 +140,7 @@ private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, b throw new SparkException("Error reply received from CacheTracker") } } - + // Registers an RDD (on master only) def registerRDD(rddId: Int, numPartitions: Int) { registeredRddIds.synchronized { @@ -151,7 +151,7 @@ private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, b } } } - + // For BlockManager.scala only def cacheLost(host: String) { communicate(MemoryCacheLost(host)) @@ -163,19 +163,20 @@ private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, b def getCacheStatus(): Seq[(String, Long, Long)] = { askTracker(GetCacheStatus).asInstanceOf[Seq[(String, Long, Long)]] } - + // For BlockManager.scala only def notifyFromBlockManager(t: AddedToCache) { communicate(t) } - + // Get a snapshot of the currently known locations def getLocationsSnapshot(): HashMap[Int, Array[List[String]]] = { askTracker(GetCacheLocations).asInstanceOf[HashMap[Int, Array[List[String]]]] } - + // Gets or computes an RDD split - def getOrCompute[T](rdd: RDD[T], split: Split, storageLevel: StorageLevel): Iterator[T] = { + def getOrCompute[T](rdd: RDD[T], split: Split, context: TaskContext, storageLevel: StorageLevel) + : Iterator[T] = { val key = "rdd_%d_%d".format(rdd.id, split.index) logInfo("Cache key is " + key) blockManager.get(key) match { @@ -217,7 +218,7 @@ private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, b // TODO: also register a listener for when it unloads logInfo("Computing partition " + split) val elements = new ArrayBuffer[Any] - elements ++= rdd.compute(split) + elements ++= rdd.compute(split, context) try { // Try to put this block in the blockManager blockManager.put(key, elements, storageLevel, true) diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index 20ff5431af30bd99caee4479e8620252c1c846e0..5ebdba0fc8a134994b748eaf89c6d71842f4f2ef 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -2,6 +2,10 @@ package spark import java.io._ import java.util.concurrent.ConcurrentHashMap +import java.util.zip.{GZIPInputStream, GZIPOutputStream} + +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet import akka.actor._ import akka.dispatch._ @@ -11,17 +15,14 @@ import akka.util.Duration import akka.util.Timeout import akka.util.duration._ -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet - -import scheduler.MapStatus +import spark.scheduler.MapStatus import spark.storage.BlockManagerId -import java.util.zip.{GZIPInputStream, GZIPOutputStream} -import util.{MetadataCleaner, TimeStampedHashMap} +import spark.util.{MetadataCleaner, TimeStampedHashMap} + private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int, requester: String) - extends MapOutputTrackerMessage + extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Actor with Logging { @@ -91,14 +92,14 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea } mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)) } - + def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) { var array = mapStatuses(shuffleId) array.synchronized { array(mapId) = status } } - + def registerMapOutputs( shuffleId: Int, statuses: Array[MapStatus], @@ -113,7 +114,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea var array = mapStatuses(shuffleId) if (array != null) { array.synchronized { - if (array(mapId).address == bmAddress) { + if (array(mapId) != null && array(mapId).address == bmAddress) { array(mapId) = null } } @@ -122,10 +123,10 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID") } } - + // Remembers which map output locations are currently being fetched on a worker val fetching = new HashSet[Int] - + // 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).orNull @@ -150,14 +151,23 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea // We won the race to fetch the output locs; do so logInfo("Doing the fetch; tracker actor = " + trackerActor) val host = System.getProperty("spark.hostname", Utils.localHostName) - val fetchedBytes = askTracker(GetMapOutputStatuses(shuffleId, host)).asInstanceOf[Array[Byte]] - val fetchedStatuses = deserializeStatuses(fetchedBytes) - - logInfo("Got the output locations") - mapStatuses.put(shuffleId, fetchedStatuses) - fetching.synchronized { - fetching -= shuffleId - fetching.notifyAll() + // This try-finally prevents hangs due to timeouts: + var fetchedStatuses: Array[MapStatus] = null + try { + val fetchedBytes = + askTracker(GetMapOutputStatuses(shuffleId, host)).asInstanceOf[Array[Byte]] + fetchedStatuses = deserializeStatuses(fetchedBytes) + logInfo("Got the output locations") + mapStatuses.put(shuffleId, fetchedStatuses) + if (fetchedStatuses.contains(null)) { + throw new FetchFailedException(null, shuffleId, -1, reduceId, + new Exception("Missing an output location for shuffle " + shuffleId)) + } + } finally { + fetching.synchronized { + fetching -= shuffleId + fetching.notifyAll() + } } return fetchedStatuses.map(s => (s.address, MapOutputTracker.decompressSize(s.compressedSizes(reduceId)))) @@ -263,8 +273,10 @@ private[spark] object MapOutputTracker { * sizes up to 35 GB with at most 10% error. */ def compressSize(size: Long): Byte = { - if (size <= 1L) { + if (size == 0) { 0 + } else if (size <= 1L) { + 1 } else { math.min(255, math.ceil(math.log(size) / math.log(LOG_BASE)).toInt).toByte } @@ -275,7 +287,7 @@ private[spark] object MapOutputTracker { */ def decompressSize(compressedSize: Byte): Long = { if (compressedSize == 0) { - 1 + 0 } else { math.pow(LOG_BASE, (compressedSize & 0xFF)).toLong } diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 1f82bd3ab861a6d1481a2bdfb95e94bb79cbb861..ec6b209932b5e17dd85743cec791744d2a5b7104 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -36,11 +36,11 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( with Serializable { /** - * Generic function to combine the elements for each key using a custom set of aggregation + * Generic function to combine the elements for each key using a custom set of aggregation * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C * Note that V and C can be different -- for example, one might group an RDD of type * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: - * + * * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) * - `mergeCombiners`, to combine two C's into a single one. @@ -119,7 +119,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** Count the number of elements for each key, and return the result to the master as a Map. */ def countByKey(): Map[K, Long] = self.map(_._1).countByValue() - /** + /** * (Experimental) Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ @@ -225,7 +225,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } } - /** + /** * Simplified version of combineByKey that hash-partitions the resulting RDD using the default * parallelism level. */ @@ -630,7 +630,8 @@ class MappedValuesRDD[K, V, U](prev: WeakReference[RDD[(K, V)]], f: V => U) override def splits = firstParent[(K, V)].splits override val partitioner = firstParent[(K, V)].partitioner - override def compute(split: Split) = firstParent[(K, V)].iterator(split).map{case (k, v) => (k, f(v))} + override def compute(split: Split, context: TaskContext) = + firstParent[(K, V)].iterator(split, context).map{ case (k, v) => (k, f(v)) } } private[spark] @@ -639,8 +640,8 @@ class FlatMappedValuesRDD[K, V, U](prev: WeakReference[RDD[(K, V)]], f: V => Tra override def splits = firstParent[(K, V)].splits override val partitioner = firstParent[(K, V)].partitioner - override def compute(split: Split) = { - firstParent[(K, V)].iterator(split).flatMap { case (k, v) => f(v).map(x => (k, x)) } + override def compute(split: Split, context: TaskContext) = { + firstParent[(K, V)].iterator(split, context).flatMap { case (k, v) => f(v).map(x => (k, x)) } } } diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/ParallelCollection.scala index 4bd9e1bd541cebf8d10e34466b58a889906206db..68416a78d0a197a80221725bd25459f875ffbf59 100644 --- a/core/src/main/scala/spark/ParallelCollection.scala +++ b/core/src/main/scala/spark/ParallelCollection.scala @@ -9,8 +9,8 @@ private[spark] class ParallelCollectionSplit[T: ClassManifest]( val slice: Int, values: Seq[T]) extends Split with Serializable { - - def iterator(): Iterator[T] = values.iterator + + def iterator: Iterator[T] = values.iterator override def hashCode(): Int = (41 * (41 + rddId) + slice).toInt @@ -40,8 +40,10 @@ private[spark] class ParallelCollection[T: ClassManifest]( override def splits = splits_.asInstanceOf[Array[Split]] - override def compute(s: Split) = s.asInstanceOf[ParallelCollectionSplit[T]].iterator - + + override def compute(s: Split, context: TaskContext) = + s.asInstanceOf[ParallelCollectionSplit[T]].iterator + override def preferredLocations(s: Split): Seq[String] = { locationPrefs.get(splits_.indexOf(s)) match { case Some(s) => s @@ -50,10 +52,11 @@ private[spark] class ParallelCollection[T: ClassManifest]( } } + private object ParallelCollection { /** * Slice a collection into numSlices sub-collections. One extra thing we do here is to treat Range - * collections specially, encoding the slices as other Ranges to minimize memory cost. This makes + * collections specially, encoding the slices as other Ranges to minimize memory cost. This makes * it efficient to run Spark over RDDs representing large sets of numbers. */ def slice[T: ClassManifest](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = { @@ -63,7 +66,7 @@ private object ParallelCollection { seq match { case r: Range.Inclusive => { val sign = if (r.step < 0) { - -1 + -1 } else { 1 } diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index fbfcfbd7040de0f5f81e109d8a6582d6414c41cf..cf3ed067735c673381488a1f15fa038069b3bf4d 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -2,15 +2,14 @@ package spark import java.io.{ObjectOutputStream, IOException, EOFException, ObjectInputStream} import java.net.URL -import java.util.concurrent.atomic.AtomicLong -import java.util.Random -import java.util.Date +import java.util.{Date, Random} import java.util.{HashMap => JHashMap} +import java.util.concurrent.atomic.AtomicLong -import scala.collection.mutable.ArrayBuffer import scala.collection.Map -import scala.collection.mutable.HashMap 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 @@ -42,12 +41,13 @@ import spark.rdd.MapPartitionsWithSplitRDD import spark.rdd.PipedRDD import spark.rdd.SampledRDD import spark.rdd.UnionRDD +import spark.rdd.ZippedRDD import spark.storage.StorageLevel import SparkContext._ /** - * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, + * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, * partitioned collection of elements that can be operated on in parallel. This class contains the * basic operations available on all RDDs, such as `map`, `filter`, and `persist`. In addition, * [[spark.PairRDDFunctions]] contains operations available only on RDDs of key-value pairs, such @@ -75,7 +75,7 @@ import SparkContext._ abstract class RDD[T: ClassManifest]( @transient var sc: SparkContext, var dependencies_ : List[Dependency[_]] - ) extends Serializable { + ) extends Serializable with Logging { def this(@transient oneParent: RDD[_]) = @@ -87,14 +87,14 @@ abstract class RDD[T: ClassManifest]( def splits: Array[Split] /** Function for computing a given partition. */ - def compute(split: Split): Iterator[T] + def compute(split: Split, context: TaskContext): Iterator[T] /** How this RDD depends on any parent RDDs. */ def dependencies: List[Dependency[_]] = dependencies_ /** Record user function generating this RDD. */ private[spark] val origin = Utils.getSparkCallSite - + /** Optionally overridden by subclasses to specify how they are partitioned. */ val partitioner: Option[Partitioner] = None @@ -106,38 +106,52 @@ abstract class RDD[T: ClassManifest]( Nil } } - + /** The [[spark.SparkContext]] that this RDD was created on. */ def context = sc private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T] - + /** A unique ID for this RDD (within its SparkContext). */ val id = sc.newRddId() - + // Variables relating to persistence private var storageLevel: StorageLevel = StorageLevel.NONE /** Returns the first parent RDD */ - protected[spark] def firstParent[U: ClassManifest] = { - dependencies.head.rdd.asInstanceOf[RDD[U]] - } + protected[spark] def firstParent[U: ClassManifest] = dependencies.head.rdd.asInstanceOf[RDD[U]] /** Returns the `i` th parent RDD */ protected[spark] def parent[U: ClassManifest](i: Int) = dependencies(i).rdd.asInstanceOf[RDD[U]] - // Variables relating to checkpointing - protected val isCheckpointable = true // override to set this to false to avoid checkpointing an RDD + ////////////////////////////////////////////////////////////////////////////// + // Checkpointing related variables + ////////////////////////////////////////////////////////////////////////////// + + // override to set this to false to avoid checkpointing an RDD + protected val isCheckpointable = true + + // set to true when an RDD is marked for checkpointing + protected var shouldCheckpoint = false + + // set to true when checkpointing is in progress + protected var isCheckpointInProgress = false + + // set to true after checkpointing is completed + protected[spark] var isCheckpointed = false - protected var shouldCheckpoint = false // set to true when an RDD is marked for checkpointing - protected var isCheckpointInProgress = false // set to true when checkpointing is in progress - protected[spark] var isCheckpointed = false // set to true after checkpointing is completed + // set to the checkpoint file after checkpointing is completed + protected[spark] var checkpointFile: String = null - protected[spark] var checkpointFile: String = null // set to the checkpoint file after checkpointing is completed - protected var checkpointRDD: RDD[T] = null // set to the HadoopRDD of the checkpoint file - protected var checkpointRDDSplits: Seq[Split] = null // set to the splits of the Hadoop RDD + // set to the HadoopRDD of the checkpoint file + protected var checkpointRDD: RDD[T] = null - // Methods available on all RDDs: + // set to the splits of the Hadoop RDD + protected var checkpointRDDSplits: Seq[Split] = null + + ////////////////////////////////////////////////////////////////////////////// + // Methods available on all RDDs + ////////////////////////////////////////////////////////////////////////////// /** * Set this RDD's storage level to persist its values across operations after the first time @@ -155,7 +169,7 @@ abstract class RDD[T: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY) - + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): RDD[T] = persist() @@ -194,10 +208,10 @@ abstract class RDD[T: ClassManifest]( } /** - * 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). In case this RDD is not marked for checkpointing, - * doCheckpoint() is called recursively on the parent RDDs. + * 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). In case this RDD is not marked for checkpointing, doCheckpoint() is called + * recursively on the parent RDDs. */ private[spark] def doCheckpoint() { val startCheckpoint = synchronized { @@ -220,7 +234,8 @@ abstract class RDD[T: ClassManifest]( rdd.shouldCheckpoint = false rdd.isCheckpointInProgress = false rdd.isCheckpointed = true - println("Done checkpointing RDD " + rdd.id + ", " + rdd + ", created RDD " + rdd.checkpointRDD.id + ", " + rdd.checkpointRDD) + logInfo("Done checkpointing RDD " + rdd.id + ", " + rdd + ", created RDD " + + rdd.checkpointRDD.id + ", " + rdd.checkpointRDD) } } else { // Recursively call doCheckpoint() to perform checkpointing on parent RDD if they are marked @@ -229,11 +244,11 @@ abstract class RDD[T: ClassManifest]( } /** - * Changes the dependencies of this RDD from its original parents to the new [[spark.rdd.HadoopRDD]] - * (`newRDD`) created from the checkpoint file. This method must ensure that all references - * to the original parent RDDs must be removed to enable the parent RDDs to be garbage - * collected. Subclasses of RDD may override this method for implementing their own changing - * logic. See [[spark.rdd.UnionRDD]] and [[spark.rdd.ShuffledRDD]] to get a better idea. + * Changes the dependencies of this RDD from its original parents to the new + * [[spark.rdd.HadoopRDD]] (`newRDD`) created from the checkpoint file. This method must ensure + * that all references to the original parent RDDs must be removed to enable the parent RDDs to + * be garbage collected. Subclasses of RDD may override this method for implementing their own + * changing logic. See [[spark.rdd.UnionRDD]] and [[spark.rdd.ShuffledRDD]] to get a better idea. */ protected def changeDependencies(newRDD: RDD[_]) { dependencies_ = List(new OneToOneDependency(newRDD)) @@ -244,19 +259,19 @@ abstract class RDD[T: ClassManifest]( * This should ''not'' be called by users directly, but is available for implementors of custom * subclasses of RDD. */ - final def iterator(split: Split): Iterator[T] = { + final def iterator(split: Split, context: TaskContext): Iterator[T] = { if (isCheckpointed) { // ASSUMPTION: Checkpoint Hadoop RDD will have same number of splits as original - checkpointRDD.iterator(checkpointRDDSplits(split.index)) + checkpointRDD.iterator(checkpointRDDSplits(split.index), context) } else if (storageLevel != StorageLevel.NONE) { - SparkEnv.get.cacheTracker.getOrCompute[T](this, split, storageLevel) + SparkEnv.get.cacheTracker.getOrCompute[T](this, split, context, storageLevel) } else { - compute(split) + compute(split, context) } } - + // Transformations (return a new RDD) - + /** * Return a new RDD by applying a function to all elements of this RDD. */ @@ -292,13 +307,13 @@ abstract class RDD[T: ClassManifest]( var multiplier = 3.0 var initialCount = count() var maxSelected = 0 - + if (initialCount > Integer.MAX_VALUE - 1) { maxSelected = Integer.MAX_VALUE - 1 } else { maxSelected = initialCount.toInt } - + if (num > initialCount) { total = maxSelected fraction = math.min(multiplier * (maxSelected + 1) / initialCount, 1.0) @@ -308,14 +323,14 @@ abstract class RDD[T: ClassManifest]( fraction = math.min(multiplier * (num + 1) / initialCount, 1.0) total = num } - + val rand = new Random(seed) var samples = this.sample(withReplacement, fraction, rand.nextInt).collect() - + while (samples.length < total) { samples = this.sample(withReplacement, fraction, rand.nextInt).collect() } - + Utils.randomizeInPlace(samples, rand).take(total) } @@ -383,8 +398,18 @@ abstract class RDD[T: ClassManifest]( * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithSplit[U: ClassManifest](f: (Int, Iterator[T]) => Iterator[U]): RDD[U] = - new MapPartitionsWithSplitRDD(this, sc.clean(f)) + def mapPartitionsWithSplit[U: ClassManifest]( + f: (Int, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = + new MapPartitionsWithSplitRDD(this, sc.clean(f), preservesPartitioning) + + /** + * Zips this RDD with another one, returning key-value pairs with the first element in each RDD, + * second element in each RDD, etc. Assumes that the two RDDs have the *same number of + * partitions* and the *same number of elements in each partition* (e.g. one was made through + * a map on the other). + */ + def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other) // Actions (launch a job to return a value to the user program) @@ -435,7 +460,7 @@ abstract class RDD[T: ClassManifest]( /** * Aggregate the elements of each partition, and then the results for all the partitions, using a - * given associative function and a neutral "zero value". The function op(t1, t2) is allowed to + * given associative function and a neutral "zero value". The function op(t1, t2) is allowed to * modify t1 and return it as its result value to avoid object allocation; however, it should not * modify t2. */ @@ -536,7 +561,7 @@ abstract class RDD[T: ClassManifest]( val evaluator = new GroupedCountEvaluator[T](splits.size, confidence) sc.runApproximateJob(this, countPartition, evaluator, timeout) } - + /** * Take the first num elements of the RDD. This currently scans the partitions *one by one*, so * it will be slow if a lot of partitions are required. In that case, use collect() to get the diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 3ccdbfe10ef7cc6c7d9250508b1952a16af55b91..70257193cf415b7aaa27e07dce8c5cbe0c77dd99 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -87,7 +87,7 @@ class SparkContext( // Set Spark master host and port system properties if (System.getProperty("spark.master.host") == null) { - System.setProperty("spark.master.host", Utils.localIpAddress()) + System.setProperty("spark.master.host", Utils.localIpAddress) } if (System.getProperty("spark.master.port") == null) { System.setProperty("spark.master.port", "0") diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 9f2b0c42c74d60bd933240af9fb88d706708a72b..41441720a7c8f81004fb897763a6d8ae2d29e5d9 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -86,10 +86,13 @@ object SparkEnv extends Logging { } val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer") - - val blockManagerMaster = new BlockManagerMaster(actorSystem, isMaster, isLocal) - val blockManager = new BlockManager(blockManagerMaster, serializer) - + + val masterIp: String = System.getProperty("spark.master.host", "localhost") + val masterPort: Int = System.getProperty("spark.master.port", "7077").toInt + val blockManagerMaster = new BlockManagerMaster( + actorSystem, isMaster, isLocal, masterIp, masterPort) + val blockManager = new BlockManager(actorSystem, blockManagerMaster, serializer) + val connectionManager = blockManager.connectionManager val broadcastManager = new BroadcastManager(isMaster) @@ -104,7 +107,7 @@ object SparkEnv extends Logging { val shuffleFetcher = instantiateClass[ShuffleFetcher]( "spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher") - + val httpFileServer = new HttpFileServer() httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index c14377d17b700c7f537840db486d610943184938..d2746b26b3ec1ff5f4766b2e53837c9a3be92423 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,3 +1,20 @@ package spark -class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long) extends Serializable +import scala.collection.mutable.ArrayBuffer + + +class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long) extends Serializable { + + @transient + val onCompleteCallbacks = new ArrayBuffer[() => Unit] + + // Add a callback function to be executed on task completion. An example use + // is for HadoopRDD to register a callback to close the input stream. + def addOnCompleteCallback(f: () => Unit) { + onCompleteCallbacks += f + } + + def executeOnCompleteCallbacks() { + onCompleteCallbacks.foreach{_()} + } +} diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 06fa559fb6c7b15e69f6a76ab746509ae87ccb9b..63e17ee4e3e77883ef440cf45ddd782c0cb8991e 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -1,12 +1,13 @@ package spark import java.io._ -import java.net.{InetAddress, URL, URI} +import java.net.{NetworkInterface, InetAddress, URL, URI} import java.util.{Locale, Random, UUID} import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ import scala.io.Source /** @@ -199,12 +200,34 @@ private object Utils extends Logging { /** * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4). */ - def localIpAddress(): String = { + lazy val localIpAddress: String = findLocalIpAddress() + + private def findLocalIpAddress(): String = { val defaultIpOverride = System.getenv("SPARK_LOCAL_IP") - if (defaultIpOverride != null) + if (defaultIpOverride != null) { defaultIpOverride - else - InetAddress.getLocalHost.getHostAddress + } else { + val address = InetAddress.getLocalHost + if (address.isLoopbackAddress) { + // 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) { + // We've found an address that looks reasonable! + logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + + " a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress + + " instead (on interface " + ni.getName + ")") + logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") + return addr.getHostAddress + } + } + logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + + " a loopback address: " + address.getHostAddress + ", but we couldn't find any" + + " external IP address!") + logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") + } + address.getHostAddress + } } private var customHostname: Option[String] = None diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 13fcee10045fb0e7736f4ea1090a6603dc70f70d..81d3a944668d007af6e84da90147ed95b499e153 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -1,16 +1,15 @@ package spark.api.java -import spark.{SparkContext, Split, RDD} +import java.util.{List => JList} +import scala.Tuple2 +import scala.collection.JavaConversions._ + +import spark.{SparkContext, Split, RDD, TaskContext} import spark.api.java.JavaPairRDD._ import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import spark.partial.{PartialResult, BoundedDouble} import spark.storage.StorageLevel -import java.util.{List => JList} - -import scala.collection.JavaConversions._ -import java.{util, lang} -import scala.Tuple2 trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def wrapRDD(rdd: RDD[T]): This @@ -24,7 +23,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** The [[spark.SparkContext]] that this RDD was created on. */ def context: SparkContext = rdd.context - + /** A unique ID for this RDD (within its SparkContext). */ def id: Int = rdd.id @@ -36,7 +35,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * This should ''not'' be called by users directly, but is available for implementors of custom * subclasses of RDD. */ - def iterator(split: Split): java.util.Iterator[T] = asJavaIterator(rdd.iterator(split)) + def iterator(split: Split, taskContext: TaskContext): java.util.Iterator[T] = + asJavaIterator(rdd.iterator(split, taskContext)) // Transformations (return a new RDD) @@ -99,7 +99,6 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { JavaRDD.fromRDD(rdd.mapPartitions(fn)(f.elementType()))(f.elementType()) } - /** * Return a new RDD by applying a function to each partition of this RDD. */ @@ -172,8 +171,18 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def pipe(command: JList[String], env: java.util.Map[String, String]): JavaRDD[String] = rdd.pipe(asScalaBuffer(command), mapAsScalaMap(env)) + /** + * Zips this RDD with another one, returning key-value pairs with the first element in each RDD, + * second element in each RDD, etc. Assumes that the two RDDs have the *same number of + * partitions* and the *same number of elements in each partition* (e.g. one was made through + * a map on the other). + */ + def zip[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] = { + JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classManifest))(classManifest, other.classManifest) + } + // Actions (launch a job to return a value to the user program) - + /** * Applies a function f to all elements of this RDD. */ @@ -190,7 +199,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { val arr: java.util.Collection[T] = rdd.collect().toSeq new java.util.ArrayList(arr) } - + /** * Reduces the elements of this RDD using the specified associative binary operator. */ @@ -198,7 +207,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Aggregate the elements of each partition, and then the results for all the partitions, using a - * given associative function and a neutral "zero value". The function op(t1, t2) is allowed to + * given associative function and a neutral "zero value". The function op(t1, t2) is allowed to * modify t1 and return it as its result value to avoid object allocation; however, it should not * modify t2. */ @@ -241,7 +250,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * combine step happens locally on the master, equivalent to running a single reduce task. */ def countByValue(): java.util.Map[T, java.lang.Long] = - mapAsJavaMap(rdd.countByValue().map((x => (x._1, new lang.Long(x._2))))) + mapAsJavaMap(rdd.countByValue().map((x => (x._1, new java.lang.Long(x._2))))) /** * (Experimental) Approximate version of countByValue(). diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala index ef27bbb5029c999925d92a8323900e8a1696a896..386f505f2a9db12482c81386bac4414a9c4f4d9e 100644 --- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala @@ -48,7 +48,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: // Used only in Workers @transient var ttGuide: TalkToGuide = null - @transient var hostAddress = Utils.localIpAddress() + @transient var hostAddress = Utils.localIpAddress @transient var listenPort = -1 @transient var guidePort = -1 diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala index fa676e90648f27b9f50a1491a37e10507fe40288..f573512835d03ce5f807847fa8d3735a2927046d 100644 --- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala @@ -36,7 +36,7 @@ extends Broadcast[T](id) with Logging with Serializable { @transient var serveMR: ServeMultipleRequests = null @transient var guideMR: GuideMultipleRequests = null - @transient var hostAddress = Utils.localIpAddress() + @transient var hostAddress = Utils.localIpAddress @transient var listenPort = -1 @transient var guidePort = -1 @@ -138,7 +138,7 @@ extends Broadcast[T](id) with Logging with Serializable { serveMR = null - hostAddress = Utils.localIpAddress() + hostAddress = Utils.localIpAddress listenPort = -1 stopBroadcast = false diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 7a1089c816a2c88b22c0f2cd3816b5dc791f5590..457122745b61847739080681da0aa429dadc0a10 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -11,8 +11,15 @@ private[spark] sealed trait DeployMessage extends Serializable // Worker to Master -private[spark] -case class RegisterWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int) +private[spark] +case class RegisterWorker( + id: String, + host: String, + port: Int, + cores: Int, + memory: Int, + webUiPort: Int, + publicAddress: String) extends DeployMessage private[spark] @@ -20,7 +27,8 @@ case class ExecutorStateChanged( jobId: String, execId: Int, state: ExecutorState, - message: Option[String]) + message: Option[String], + exitStatus: Option[Int]) extends DeployMessage // Master to Worker @@ -51,7 +59,8 @@ private[spark] case class ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) private[spark] -case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String]) +case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String], + exitStatus: Option[Int]) private[spark] case class JobKilled(message: String) diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 8b2a71add53b85c41a7b02044c3089ce9b48500c..4211d805967a9087e19a7f0285d9efdad776913c 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -35,11 +35,15 @@ class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int) /* Start the Slaves */ for (slaveNum <- 1 to numSlaves) { + /* We can pretend to test distributed stuff by giving the slaves distinct hostnames. + All of 127/8 should be a loopback, we use 127.100.*.* in hopes that it is + sufficiently distinctive. */ + val slaveIpAddress = "127.100.0." + (slaveNum % 256) val (actorSystem, boundPort) = - AkkaUtils.createActorSystem("sparkWorker" + slaveNum, localIpAddress, 0) + AkkaUtils.createActorSystem("sparkWorker" + slaveNum, slaveIpAddress, 0) slaveActorSystems += actorSystem val actor = actorSystem.actorOf( - Props(new Worker(localIpAddress, boundPort, 0, coresPerSlave, memoryPerSlave, masterUrl)), + Props(new Worker(slaveIpAddress, boundPort, 0, coresPerSlave, memoryPerSlave, masterUrl)), name = "Worker") slaveActors += actor } diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index c57a1d33e91f282302876a637f69cbada39597c2..90fe9508cdba9b5df0b438ee4e91447915c2c6fd 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -66,12 +66,12 @@ private[spark] class Client( logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, host, cores)) listener.executorAdded(fullId, workerId, host, cores, memory) - case ExecutorUpdated(id, state, message) => + case ExecutorUpdated(id, state, message, exitStatus) => val fullId = jobId + "/" + id val messageText = message.map(s => " (" + s + ")").getOrElse("") logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText)) if (ExecutorState.isFinished(state)) { - listener.executorRemoved(fullId, message.getOrElse("")) + listener.executorRemoved(fullId, message.getOrElse(""), exitStatus) } case Terminated(actor_) if actor_ == master => diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala index a8fa982085dd25eae4a24f00ea8c37c47857d155..da6abcc9c26c083482eaaac5f002e2151d4803b1 100644 --- a/core/src/main/scala/spark/deploy/client/ClientListener.scala +++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala @@ -14,5 +14,5 @@ private[spark] trait ClientListener { def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int): Unit - def executorRemoved(id: String, message: String): Unit + def executorRemoved(id: String, message: String, exitStatus: Option[Int]): Unit } diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index bf0e7428bac3c1f8cb9cf83e4e2ba5c6bf038ea5..57a7e123b78a5449802469eb9bab7bb21753020d 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -18,12 +18,12 @@ private[spark] object TestClient { def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int) {} - def executorRemoved(id: String, message: String) {} + def executorRemoved(id: String, message: String, exitStatus: Option[Int]) {} } def main(args: Array[String]) { val url = args(0) - val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress(), 0) + val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) val desc = new JobDescription( "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map())) val listener = new TestListener diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 7e5cd6b1719ec46c18499f76fc80b6e0e9c11d1b..6ecebe626a7cef9d7be27fef71c84a6f3f76c8c7 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -31,6 +31,11 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor val waitingJobs = new ArrayBuffer[JobInfo] val completedJobs = new ArrayBuffer[JobInfo] + val masterPublicAddress = { + val envVar = System.getenv("SPARK_PUBLIC_DNS") + if (envVar != null) envVar else ip + } + // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each job // among all the nodes) instead of trying to consolidate each job onto a small # of nodes. @@ -55,15 +60,15 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } override def receive = { - case RegisterWorker(id, host, workerPort, cores, memory, worker_webUiPort) => { + case RegisterWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) => { logInfo("Registering worker %s:%d with %d cores, %s RAM".format( host, workerPort, cores, Utils.memoryMegabytesToString(memory))) if (idToWorker.contains(id)) { sender ! RegisterWorkerFailed("Duplicate worker ID") } else { - addWorker(id, host, workerPort, cores, memory, worker_webUiPort) + addWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) context.watch(sender) // This doesn't work with remote actors but helps for testing - sender ! RegisteredWorker("http://" + ip + ":" + webUiPort) + sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUiPort) schedule() } } @@ -78,12 +83,12 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor schedule() } - case ExecutorStateChanged(jobId, execId, state, message) => { + case ExecutorStateChanged(jobId, execId, state, message, exitStatus) => { val execOption = idToJob.get(jobId).flatMap(job => job.executors.get(execId)) execOption match { case Some(exec) => { exec.state = state - exec.job.actor ! ExecutorUpdated(execId, state, message) + exec.job.actor ! ExecutorUpdated(execId, state, message, exitStatus) if (ExecutorState.isFinished(state)) { val jobInfo = idToJob(jobId) // Remove this executor from the worker and job @@ -151,7 +156,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor if (spreadOutJobs) { // Try to spread out each job among all the nodes, until it has all its cores for (job <- waitingJobs if job.coresLeft > 0) { - val usableWorkers = workers.toArray.filter(canUse(job, _)).sortBy(_.coresFree).reverse + val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) + .filter(canUse(job, _)).sortBy(_.coresFree).reverse val numUsable = usableWorkers.length val assigned = new Array[Int](numUsable) // Number of cores to give on each node var toAssign = math.min(job.coresLeft, usableWorkers.map(_.coresFree).sum) @@ -196,8 +202,11 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor exec.job.actor ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory) } - def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int): WorkerInfo = { - val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort) + def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int, + publicAddress: String): WorkerInfo = { + // There may be one or more refs to dead workers on this same node (w/ different ID's), remove them. + workers.filter(w => (w.host == host) && (w.state == WorkerState.DEAD)).foreach(workers -= _) + val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress) workers += worker idToWorker(worker.id) = worker actorToWorker(sender) = worker @@ -207,12 +216,12 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor def removeWorker(worker: WorkerInfo) { logInfo("Removing worker " + worker.id + " on " + worker.host + ":" + worker.port) - workers -= worker + worker.setState(WorkerState.DEAD) idToWorker -= worker.id actorToWorker -= worker.actor addressToWorker -= worker.actor.path.address for (exec <- worker.executors.values) { - exec.job.actor ! ExecutorStateChanged(exec.job.id, exec.id, ExecutorState.LOST, None) + exec.job.actor ! ExecutorStateChanged(exec.job.id, exec.id, ExecutorState.LOST, None, None) exec.job.executors -= exec.id } } diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala index 706b1453aa6fadd5f23cf6502171c1df1d85d83a..5a7f5fef8a546812d20719af8d2a0f3dcab1af29 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala @@ -10,10 +10,11 @@ private[spark] class WorkerInfo( val cores: Int, val memory: Int, val actor: ActorRef, - val webUiPort: Int) { + val webUiPort: Int, + val publicAddress: String) { var executors = new mutable.HashMap[String, ExecutorInfo] // fullId => info - + var state: WorkerState.Value = WorkerState.ALIVE var coresUsed = 0 var memoryUsed = 0 @@ -37,8 +38,12 @@ private[spark] class WorkerInfo( def hasExecutor(job: JobInfo): Boolean = { executors.values.exists(_.job == job) } - + def webUiAddress : String = { - "http://" + this.host + ":" + this.webUiPort + "http://" + this.publicAddress + ":" + this.webUiPort + } + + def setState(state: WorkerState.Value) = { + this.state = state } } diff --git a/core/src/main/scala/spark/deploy/master/WorkerState.scala b/core/src/main/scala/spark/deploy/master/WorkerState.scala new file mode 100644 index 0000000000000000000000000000000000000000..0bf35014c806ccc9f61a135b97a5cfc6e791db42 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/WorkerState.scala @@ -0,0 +1,7 @@ +package spark.deploy.master + +private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { + type WorkerState = Value + + val ALIVE, DEAD, DECOMMISSIONED = Value +} diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 07ae7bca78e9b582fe7892af194866ae2824759a..beceb55ecdf54750016af497b0689b5a37191a67 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -60,7 +60,7 @@ private[spark] class ExecutorRunner( process.destroy() process.waitFor() } - worker ! ExecutorStateChanged(jobId, execId, ExecutorState.KILLED, None) + worker ! ExecutorStateChanged(jobId, execId, ExecutorState.KILLED, None, None) Runtime.getRuntime.removeShutdownHook(shutdownHook) } } @@ -134,7 +134,8 @@ private[spark] class ExecutorRunner( // times on the same machine. val exitCode = process.waitFor() val message = "Command exited with code " + exitCode - worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message)) + worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message), + Some(exitCode)) } catch { case interrupted: InterruptedException => logInfo("Runner thread for executor " + fullId + " interrupted") @@ -145,7 +146,7 @@ private[spark] class ExecutorRunner( process.destroy() } val message = e.getClass + ": " + e.getMessage - worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message)) + worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message), None) } } } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 67d41dda29f1d65e8424f355a44e90671e6416f2..7c9e588ea2d3277bb85a800c429fdc6179da1727 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -36,6 +36,10 @@ private[spark] class Worker( var workDir: File = null val executors = new HashMap[String, ExecutorRunner] val finishedExecutors = new HashMap[String, ExecutorRunner] + val publicAddress = { + val envVar = System.getenv("SPARK_PUBLIC_DNS") + if (envVar != null) envVar else ip + } var coresUsed = 0 var memoryUsed = 0 @@ -79,7 +83,7 @@ private[spark] class Worker( val akkaUrl = "akka://spark@%s:%s/user/Master".format(masterHost, masterPort) try { master = context.actorFor(akkaUrl) - master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort) + master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort, publicAddress) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { @@ -123,10 +127,10 @@ private[spark] class Worker( manager.start() coresUsed += cores_ memoryUsed += memory_ - master ! ExecutorStateChanged(jobId, execId, ExecutorState.RUNNING, None) + master ! ExecutorStateChanged(jobId, execId, ExecutorState.RUNNING, None, None) - case ExecutorStateChanged(jobId, execId, state, message) => - master ! ExecutorStateChanged(jobId, execId, state, message) + case ExecutorStateChanged(jobId, execId, state, message, exitStatus) => + master ! ExecutorStateChanged(jobId, execId, state, message, exitStatus) val fullId = jobId + "/" + execId if (ExecutorState.isFinished(state)) { val executor = executors(fullId) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index cb29a6b8b46e2d78462dc170ee80342d1ace82d3..2552958d27e37a0aee8afddba93e40c2fb83f4bd 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -50,9 +50,14 @@ private[spark] class Executor extends Logging { override def uncaughtException(thread: Thread, exception: Throwable) { try { logError("Uncaught exception in thread " + thread, exception) - System.exit(1) + if (exception.isInstanceOf[OutOfMemoryError]) { + System.exit(ExecutorExitCode.OOM) + } else { + System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + } } catch { - case t: Throwable => System.exit(2) + case oom: OutOfMemoryError => System.exit(ExecutorExitCode.OOM) + case t: Throwable => System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) } } } diff --git a/core/src/main/scala/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/spark/executor/ExecutorExitCode.scala new file mode 100644 index 0000000000000000000000000000000000000000..fd76029cb34446e6c33f1c53cf593db1089d12b1 --- /dev/null +++ b/core/src/main/scala/spark/executor/ExecutorExitCode.scala @@ -0,0 +1,43 @@ +package spark.executor + +/** + * These are exit codes that executors should use to provide the master with information about + * executor failures assuming that cluster management framework can capture the exit codes (but + * perhaps not log files). The exit code constants here are chosen to be unlikely to conflict + * with "natural" exit statuses that may be caused by the JVM or user code. In particular, + * exit codes 128+ arise on some Unix-likes as a result of signals, and it appears that the + * OpenJDK JVM may use exit code 1 in some of its own "last chance" code. + */ +private[spark] +object ExecutorExitCode { + /** The default uncaught exception handler was reached. */ + val UNCAUGHT_EXCEPTION = 50 + + /** The default uncaught exception handler was called and an exception was encountered while + logging the exception. */ + val UNCAUGHT_EXCEPTION_TWICE = 51 + + /** The default uncaught exception handler was reached, and the uncaught exception was an + OutOfMemoryError. */ + val OOM = 52 + + /** DiskStore failed to create a local temporary directory after many attempts. */ + val DISK_STORE_FAILED_TO_CREATE_DIR = 53 + + def explainExitCode(exitCode: Int): String = { + exitCode match { + case UNCAUGHT_EXCEPTION => "Uncaught exception" + case UNCAUGHT_EXCEPTION_TWICE => "Uncaught exception, and logging the exception failed" + case OOM => "OutOfMemoryError" + case DISK_STORE_FAILED_TO_CREATE_DIR => + "Failed to create local directory (bad spark.local.dir?)" + case _ => + "Unknown executor exit code (" + exitCode + ")" + ( + if (exitCode > 128) + " (died from signal " + (exitCode - 128) + "?)" + else + "" + ) + } + } +} diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index f4c3f99011d42e178fb22171e6c7731a8acac769..61bc5c90bab5151c8d4fee364b3ddd949df877f4 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -2,11 +2,8 @@ package spark.rdd import scala.collection.mutable.HashMap -import spark.Dependency -import spark.RDD -import spark.SparkContext -import spark.SparkEnv -import spark.Split +import spark.{RDD, SparkContext, SparkEnv, Split, TaskContext} + private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split { val index = idx @@ -19,24 +16,24 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St @transient val splits_ = (0 until blockIds.size).map(i => { new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split] - }).toArray - - @transient + }).toArray + + @transient lazy val locations_ = { - val blockManager = SparkEnv.get.blockManager + val blockManager = SparkEnv.get.blockManager /*val locations = blockIds.map(id => blockManager.getLocations(id))*/ - val locations = blockManager.getLocations(blockIds) + val locations = blockManager.getLocations(blockIds) HashMap(blockIds.zip(locations):_*) } override def splits = splits_ - override def compute(split: Split): Iterator[T] = { - val blockManager = SparkEnv.get.blockManager + override def compute(split: Split, context: TaskContext): Iterator[T] = { + val blockManager = SparkEnv.get.blockManager val blockId = split.asInstanceOf[BlockRDDSplit].blockId blockManager.get(blockId) match { case Some(block) => block.asInstanceOf[Iterator[T]] - case None => + case None => throw new Exception("Could not compute split, block " + blockId + " not found") } } diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 458ad38d5592d56eda6d051a03afa486038420da..bc11b60e052ac65a94937f1c6f06aeb3ffbba1d5 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -1,8 +1,10 @@ package spark.rdd -import spark._ import java.lang.ref.WeakReference +import spark.{OneToOneDependency, NarrowDependency, RDD, SparkContext, Split, TaskContext} + + private[spark] class CartesianSplit(idx: Int, val s1: Split, val s2: Split) extends Split with Serializable { override val index: Int = idx @@ -40,9 +42,10 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( } } - override def compute(split: Split) = { + override def compute(split: Split, context: TaskContext) = { val currSplit = split.asInstanceOf[CartesianSplit] - for (x <- rdd1.iterator(currSplit.s1); y <- rdd2.iterator(currSplit.s2)) yield (x, y) + for (x <- rdd1.iterator(currSplit.s1, context); + y <- rdd2.iterator(currSplit.s2, context)) yield (x, y) } var deps_ = List( diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 94ef1b56e891ab3074ed5f366a133b03900db0fc..ef8673909bd8a5bc553fe3c7f9f0724cdc40be60 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -1,21 +1,18 @@ package spark.rdd +import java.io.{ObjectOutputStream, IOException} + import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import spark.Aggregator -import spark.Dependency -import spark.Logging -import spark.OneToOneDependency -import spark.Partitioner -import spark.RDD -import spark.ShuffleDependency -import spark.SparkEnv -import spark.Split -import java.io.{ObjectOutputStream, IOException} +import spark.{Aggregator, Logging, Partitioner, RDD, SparkEnv, Split, TaskContext} +import spark.{Dependency, OneToOneDependency, ShuffleDependency} + private[spark] sealed trait CoGroupSplitDep extends Serializable -private[spark] case class NarrowCoGroupSplitDep(rdd: RDD[_], splitIndex: Int, var split: Split = null) + +private[spark] +case class NarrowCoGroupSplitDep(rdd: RDD[_], splitIndex: Int, var split: Split = null) extends CoGroupSplitDep { @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { @@ -26,9 +23,10 @@ private[spark] case class NarrowCoGroupSplitDep(rdd: RDD[_], splitIndex: Int, va } } } + private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep -private[spark] +private[spark] class CoGroupSplit(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Split with Serializable { override val index: Int = idx override def hashCode(): Int = idx @@ -41,9 +39,10 @@ private[spark] class CoGroupAggregator { (b1, b2) => b1 ++ b2 }) with Serializable + 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 @@ -81,10 +80,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) } override def splits = splits_ - + override val partitioner = Some(part) - - override def compute(s: Split): Iterator[(K, Seq[Seq[_]])] = { + + override def compute(s: Split, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { val split = s.asInstanceOf[CoGroupSplit] val numRdds = split.deps.size val map = new HashMap[K, Seq[ArrayBuffer[Any]]] @@ -94,7 +93,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, itsSplitIndex, itsSplit) => { // Read them from the parent - for ((k, v) <- rdd.iterator(itsSplit)) { + for ((k, v) <- rdd.iterator(itsSplit, context)) { getSeq(k.asInstanceOf[K])(depNum) += v } } diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 5b5f72ddeb5d32717dbebf15029334744c7505f1..c5e2300d2638c2cbdbdecd7531e6b356036fb57b 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -1,8 +1,10 @@ package spark.rdd -import spark._ import java.lang.ref.WeakReference +import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Split, TaskContext} + + private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) extends Split /** @@ -33,9 +35,9 @@ class CoalescedRDD[T: ClassManifest]( override def splits = splits_ - override def compute(split: Split): Iterator[T] = { - split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap { - parentSplit => firstParent[T].iterator(parentSplit) + override def compute(split: Split, context: TaskContext): Iterator[T] = { + split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap { parentSplit => + firstParent[T].iterator(parentSplit, context) } } diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala index 1370cf6faf8e29aea49b312ac4faf12cc42be09f..70c4be7903a0aabc4adf01a4d6396decfc5cf9a7 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala @@ -1,16 +1,14 @@ package spark.rdd -import spark.OneToOneDependency -import spark.RDD -import spark.Split import java.lang.ref.WeakReference +import spark.{OneToOneDependency, RDD, Split, TaskContext} + private[spark] -class FilteredRDD[T: ClassManifest]( - prev: WeakReference[RDD[T]], - f: T => Boolean) +class FilteredRDD[T: ClassManifest](prev: WeakReference[RDD[T]], f: T => Boolean) extends RDD[T](prev.get) { override def splits = firstParent[T].splits - override def compute(split: Split) = firstParent[T].iterator(split).filter(f) + override def compute(split: Split, context: TaskContext) = + firstParent[T].iterator(split, context).filter(f) } \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala index 6b2cc67568e0a251f21c6e4c85b9e21f99c8219d..1ebbb4c9bd70a7ba26eb4b2d7c266a4093f0a6ee 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala @@ -1,16 +1,17 @@ package spark.rdd -import spark.OneToOneDependency -import spark.RDD -import spark.Split import java.lang.ref.WeakReference +import spark.{RDD, Split, TaskContext} + + private[spark] class FlatMappedRDD[U: ClassManifest, T: ClassManifest]( prev: WeakReference[RDD[T]], f: T => TraversableOnce[U]) extends RDD[U](prev.get) { - + override def splits = firstParent[T].splits - override def compute(split: Split) = firstParent[T].iterator(split).flatMap(f) + override def compute(split: Split, context: TaskContext) = + firstParent[T].iterator(split, context).flatMap(f) } diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala index 0f0b6ab0ffd3ed6505e8371560d752536c900409..43661ae3f8c99b96bec0e5e1def3efa728a78e87 100644 --- a/core/src/main/scala/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala @@ -1,13 +1,13 @@ package spark.rdd -import spark.OneToOneDependency -import spark.RDD -import spark.Split import java.lang.ref.WeakReference +import spark.{RDD, Split, TaskContext} + private[spark] class GlommedRDD[T: ClassManifest](prev: WeakReference[RDD[T]]) extends RDD[Array[T]](prev.get) { override def splits = firstParent[T].splits - override def compute(split: Split) = Array(firstParent[T].iterator(split).toArray).iterator -} \ No newline at end of file + override def compute(split: Split, context: TaskContext) = + Array(firstParent[T].iterator(split, context).toArray).iterator +} diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 19ed56d9c0a19bde634c471ac0a4a13c5d39f3ce..7b5f8ac3e981aa13f9efeec3843cd3e052d4653c 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -15,19 +15,16 @@ import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.util.ReflectionUtils -import spark.Dependency -import spark.RDD -import spark.SerializableWritable -import spark.SparkContext -import spark.Split +import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskContext} -/** + +/** * 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 { - + val inputSplit = new SerializableWritable[InputSplit](s) override def hashCode(): Int = (41 * (41 + rddId) + idx).toInt @@ -47,10 +44,10 @@ class HadoopRDD[K, V]( valueClass: Class[V], minSplits: Int) 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)) - + @transient val splits_ : Array[Split] = { val inputFormat = createInputFormat(conf) @@ -69,7 +66,7 @@ class HadoopRDD[K, V]( override def splits = splits_ - override def compute(theSplit: Split) = new Iterator[(K, V)] { + override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[HadoopSplit] var reader: RecordReader[K, V] = null @@ -77,6 +74,9 @@ class HadoopRDD[K, V]( val fmt = createInputFormat(conf) reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) + // Register an on-task-completion callback to close the input stream. + context.addOnCompleteCallback(() => reader.close()) + val key: K = reader.createKey() val value: V = reader.createValue() var gotNext = false diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala index b04f56cfcc6790e67d38e9593591575ec578e8f4..991f4be73f347ecebac41baabe410ff306d220e8 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala @@ -1,10 +1,14 @@ package spark.rdd + import spark.OneToOneDependency import spark.RDD import spark.Split import java.lang.ref.WeakReference +import spark.{RDD, Split, TaskContext} + + private[spark] class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( prev: WeakReference[RDD[T]], @@ -13,7 +17,8 @@ class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( extends RDD[U](prev.get) { override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None - + override def splits = firstParent[T].splits - override def compute(split: Split) = f(firstParent[T].iterator(split)) + override def compute(split: Split, context: TaskContext) = + f(firstParent[T].iterator(split, context)) } \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala index 7a4b6ffb0310e1d894fdb9dcc19e119b596a1abb..e2e7753cded75a78f4a35fc92f3776621a0e0aa0 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala @@ -1,10 +1,10 @@ package spark.rdd -import spark.OneToOneDependency -import spark.RDD -import spark.Split import java.lang.ref.WeakReference +import spark.{RDD, Split, TaskContext} + + /** * A variant of the MapPartitionsRDD that passes the split index into the * closure. This can be used to generate or collect partition specific @@ -13,9 +13,12 @@ import java.lang.ref.WeakReference private[spark] class MapPartitionsWithSplitRDD[U: ClassManifest, T: ClassManifest]( prev: WeakReference[RDD[T]], - f: (Int, Iterator[T]) => Iterator[U]) + f: (Int, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean) extends RDD[U](prev.get) { + override val partitioner = if (preservesPartitioning) prev.get.partitioner else None override def splits = firstParent[T].splits - override def compute(split: Split) = f(split.index, firstParent[T].iterator(split)) + override def compute(split: Split, context: TaskContext) = + f(split.index, firstParent[T].iterator(split, context)) } \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala index 8fa1872e0a0e00d6fc993964e0df429b38ec775a..986cf35291ad35ea2e0a0605d238486a953e30cf 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedRDD.scala @@ -1,10 +1,10 @@ package spark.rdd -import spark.OneToOneDependency -import spark.RDD -import spark.Split import java.lang.ref.WeakReference +import spark.{RDD, Split, TaskContext} + + private[spark] class MappedRDD[U: ClassManifest, T: ClassManifest]( prev: WeakReference[RDD[T]], @@ -12,5 +12,6 @@ class MappedRDD[U: ClassManifest, T: ClassManifest]( extends RDD[U](prev.get) { override def splits = firstParent[T].splits - override def compute(split: Split) = firstParent[T].iterator(split).map(f) + override def compute(split: Split, context: TaskContext) = + firstParent[T].iterator(split, context).map(f) } \ No newline at end of file diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 2875abb2db972ec768e9b0e76ea0fb0a4b76f4e4..c7cc8d468533e5f39c5ba7bb34625992649047d3 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -1,22 +1,19 @@ package spark.rdd +import java.text.SimpleDateFormat +import java.util.Date + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import java.util.Date -import java.text.SimpleDateFormat +import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskContext} -import spark.Dependency -import spark.RDD -import spark.SerializableWritable -import spark.SparkContext -import spark.Split -private[spark] +private[spark] class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) extends Split { - + val serializableHadoopSplit = new SerializableWritable(rawSplit) override def hashCode(): Int = (41 * (41 + rddId) + index) @@ -30,7 +27,7 @@ class NewHadoopRDD[K, V]( @transient conf: Configuration) extends RDD[(K, V)](sc, Nil) with HadoopMapReduceUtil { - + // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it val confBroadcast = sc.broadcast(new SerializableWritable(conf)) // private val serializableConf = new SerializableWritable(conf) @@ -57,15 +54,19 @@ class NewHadoopRDD[K, V]( override def splits = splits_ - override def compute(theSplit: Split) = new Iterator[(K, V)] { + override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[NewHadoopSplit] val conf = confBroadcast.value.value val attemptId = new TaskAttemptID(jobtrackerId, id, true, split.index, 0) - val context = newTaskAttemptContext(conf, attemptId) + val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance - val reader = format.createRecordReader(split.serializableHadoopSplit.value, context) - reader.initialize(split.serializableHadoopSplit.value, context) - + val reader = format.createRecordReader( + split.serializableHadoopSplit.value, hadoopAttemptContext) + reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) + + // Register an on-task-completion callback to close the input stream. + context.addOnCompleteCallback(() => reader.close()) + var havePair = false var finished = false @@ -73,9 +74,6 @@ class NewHadoopRDD[K, V]( if (!finished && !havePair) { finished = !reader.nextKeyValue havePair = !finished - if (finished) { - reader.close() - } } !finished } diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index d9293a9d1a91378ca644665c26688a56b8e7f1fc..076c6a64a0531de760da1fdef034d097d3c77a12 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -1,6 +1,7 @@ package spark.rdd import java.io.PrintWriter +import java.lang.ref.WeakReference import java.util.StringTokenizer import scala.collection.Map @@ -8,11 +9,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.io.Source -import spark.OneToOneDependency -import spark.RDD -import spark.SparkEnv -import spark.Split -import java.lang.ref.WeakReference +import spark.{RDD, SparkEnv, Split, TaskContext} /** @@ -33,12 +30,12 @@ class PipedRDD[T: ClassManifest]( override def splits = firstParent[T].splits - override def compute(split: Split): Iterator[String] = { + override def compute(split: Split, context: TaskContext): Iterator[String] = { val pb = new ProcessBuilder(command) // Add the environmental variables to the process. val currentEnvVars = pb.environment() envVars.foreach { case (variable, value) => currentEnvVars.put(variable, value) } - + val proc = pb.start() val env = SparkEnv.get @@ -56,7 +53,7 @@ class PipedRDD[T: ClassManifest]( override def run() { SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) - for (elem <- firstParent[T].iterator(split)) { + for (elem <- firstParent[T].iterator(split, context)) { out.println(elem) } out.close() diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index f273f257f838df925d909c6322ac3894fae9154b..0dc83c127fb26a8048bbc20fbb531fec8301e660 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -1,13 +1,12 @@ package spark.rdd +import java.lang.ref.WeakReference import java.util.Random + import cern.jet.random.Poisson import cern.jet.random.engine.DRand -import spark.RDD -import spark.OneToOneDependency -import spark.Split -import java.lang.ref.WeakReference +import spark.{RDD, Split, TaskContext} private[spark] class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Serializable { @@ -16,7 +15,7 @@ class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Seriali class SampledRDD[T: ClassManifest]( prev: WeakReference[RDD[T]], - withReplacement: Boolean, + withReplacement: Boolean, frac: Double, seed: Int) extends RDD[T](prev.get) { @@ -32,13 +31,13 @@ class SampledRDD[T: ClassManifest]( override def preferredLocations(split: Split) = firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDSplit].prev) - override def compute(splitIn: Split) = { + override def compute(splitIn: Split, context: TaskContext) = { val split = splitIn.asInstanceOf[SampledRDDSplit] if (withReplacement) { // 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)) - firstParent[T].iterator(split.prev).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 +47,7 @@ class SampledRDD[T: ClassManifest]( } } else { // Sampling without replacement val rand = new Random(split.seed) - firstParent[T].iterator(split.prev).filter(x => (rand.nextDouble <= frac)) + firstParent[T].iterator(split.prev, context).filter(x => (rand.nextDouble <= frac)) } } } diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 31774585f44ee00a845410436297a9750fb7ac84..7d592ffc5eddeb9b213c844c7507c56b5f0a8d82 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -1,12 +1,10 @@ package spark.rdd -import spark.Partitioner -import spark.RDD -import spark.ShuffleDependency -import spark.SparkEnv -import spark.Split import java.lang.ref.WeakReference +import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Split, TaskContext} + + private[spark] class ShuffledRDDSplit(val idx: Int) extends Split { override val index = idx override def hashCode(): Int = idx @@ -31,7 +29,7 @@ class ShuffledRDD[K, V]( override def splits = splits_ - override def compute(split: Split): Iterator[(K, V)] = { + override def compute(split: Split, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index) } diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 643a174160dc0e1bd12359c8fe0ea21434e4b42b..21965d3f5d282b53d8ff24e73f9218c02502b7d4 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -1,18 +1,18 @@ package spark.rdd +import java.lang.ref.WeakReference import scala.collection.mutable.ArrayBuffer +import spark.{Dependency, OneToOneDependency, RangeDependency, RDD, SparkContext, Split, TaskContext} -import spark._ -import java.lang.ref.WeakReference private[spark] class UnionSplit[T: ClassManifest]( - idx: Int, + idx: Int, rdd: RDD[T], split: Split) extends Split with Serializable { - - def iterator() = rdd.iterator(split) + + def iterator(context: TaskContext) = rdd.iterator(split, context) def preferredLocations() = rdd.preferredLocations(split) override val index: Int = idx } @@ -20,7 +20,7 @@ private[spark] class UnionSplit[T: ClassManifest]( class UnionRDD[T: ClassManifest]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) - extends RDD[T](sc, Nil) { // Nil, so the dependencies_ var does not refer to parent RDDs + extends RDD[T](sc, Nil) { // Nil, so the dependencies_ var does not refer to parent RDDs @transient var splits_ : Array[Split] = { @@ -47,7 +47,8 @@ class UnionRDD[T: ClassManifest]( override def dependencies = deps_ - override def compute(s: Split): Iterator[T] = s.asInstanceOf[UnionSplit[T]].iterator() + override def compute(s: Split, context: TaskContext): Iterator[T] = + s.asInstanceOf[UnionSplit[T]].iterator(context) override def preferredLocations(s: Split): Seq[String] = { if (isCheckpointed) { diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala new file mode 100644 index 0000000000000000000000000000000000000000..33b64e2d249a5a04d77bd37baea478095d28da6e --- /dev/null +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -0,0 +1,55 @@ +package spark.rdd + +import spark.{OneToOneDependency, RDD, SparkContext, Split, TaskContext} + + +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)) + + override val index: Int = idx +} + +class ZippedRDD[T: ClassManifest, U: ClassManifest]( + sc: SparkContext, + @transient rdd1: RDD[T], + @transient rdd2: RDD[U]) + extends RDD[(T, U)](sc, Nil) + with Serializable { + + // TODO: FIX THIS. + + @transient + val 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 + } + + override def splits = splits_ + + @transient + override val dependencies = List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2)) + + override def compute(s: Split, context: TaskContext): Iterator[(T, U)] = + s.asInstanceOf[ZippedSplit[T, U]].iterator(context) + + override def preferredLocations(s: Split): Seq[String] = + s.asInstanceOf[ZippedSplit[T, U]].preferredLocations() +} diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 4b2570fa2bccab817c9036847d038e450b0078de..9387ba19a3956066cace388a533d96cfa489fc76 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -17,8 +17,8 @@ import spark.storage.BlockManagerId import util.{MetadataCleaner, TimeStampedHashMap} /** - * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for - * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal + * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for + * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal * schedule to run the job. Subclasses only need to implement the code to send a task to the cluster * and to report fetch failures (the submitTasks method, and code to add CompletionEvents). */ @@ -74,7 +74,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val deadHosts = new HashSet[String] // TODO: The code currently assumes these can't come back; // that's not going to be a realistic assumption in general - + 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 @@ -97,7 +97,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { cacheLocs(rdd.id) } - + def updateCacheLocs() { cacheLocs = cacheTracker.getLocationsSnapshot() } @@ -329,7 +329,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val rdd = job.finalStage.rdd val split = rdd.splits(job.partitions(0)) val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0) - val result = job.func(taskContext, rdd.iterator(split)) + val result = job.func(taskContext, rdd.iterator(split, taskContext)) + taskContext.executeOnCompleteCallbacks() job.listener.taskSucceeded(0, result) } catch { case e: Exception => @@ -356,7 +357,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with } } } - + def submitMissingTasks(stage: Stage) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry @@ -398,7 +399,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with val task = event.task val stage = idToStage(task.stageId) event.reason match { - case Success => + case Success => logInfo("Completed " + task) if (event.accumUpdates != null) { Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted @@ -482,8 +483,10 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with ") for resubmision due to a fetch failure") // Mark the map whose fetch failed as broken in the map stage val mapStage = shuffleToMapStage(shuffleId) - mapStage.removeOutputLoc(mapId, bmAddress) - mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + if (mapId != -1) { + mapStage.removeOutputLoc(mapId, bmAddress) + mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + } logInfo("The failed fetch was from " + mapStage + " (" + mapStage.origin + "); marking it for resubmission") failed += mapStage @@ -520,7 +523,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with updateCacheLocs() } } - + /** * Aborts all jobs depending on a particular Stage. This is called in response to a task set * being cancelled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 2ebd4075a26363cfe6c4df6f9c6c2aa2515ff429..e492279b4ec6444dfc793f0a0a20309c60b8c399 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -10,12 +10,14 @@ private[spark] class ResultTask[T, U]( @transient locs: Seq[String], val outputId: Int) extends Task[U](stageId) { - + val split = rdd.splits(partition) override def run(attemptId: Long): U = { val context = new TaskContext(stageId, partition, attemptId) - func(context, rdd.iterator(split)) + val result = func(context, rdd.iterator(split, context)) + context.executeOnCompleteCallbacks() + result } override def preferredLocations: Seq[String] = locs diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 683f5ebec3c4f28610f93855946dc3463fdfcfbf..7fdc178d4b6c66dd0897273d8cd72337cdc76cfa 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -73,19 +73,19 @@ private[spark] object ShuffleMapTask { private[spark] class ShuffleMapTask( stageId: Int, - var rdd: RDD[_], + var rdd: RDD[_], var dep: ShuffleDependency[_,_], - var partition: Int, + var partition: Int, @transient var locs: Seq[String]) extends Task[MapStatus](stageId) with Externalizable with Logging { def this() = this(0, null, null, 0, null) - + var split = if (rdd == null) { - null - } else { + null + } else { rdd.splits(partition) } @@ -116,9 +116,11 @@ private[spark] class ShuffleMapTask( val numOutputSplits = dep.partitioner.numPartitions val partitioner = dep.partitioner + val taskContext = new TaskContext(stageId, partition, attemptId) + // Partition the map output. val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) - for (elem <- rdd.iterator(split)) { + for (elem <- rdd.iterator(split, taskContext)) { val pair = elem.asInstanceOf[(Any, Any)] val bucketId = partitioner.getPartition(pair._1) buckets(bucketId) += pair @@ -136,6 +138,9 @@ private[spark] class ShuffleMapTask( compressedSizes(i) = MapOutputTracker.compressSize(size) } + // Execute the callbacks on task completion. + taskContext.executeOnCompleteCallbacks() + return new MapStatus(blockManager.blockManagerId, compressedSizes) } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index f5e852d203b496ff988678a7ee3316596f6d7dd2..20f6e65020f54d87418ffb8a1441a886f759c754 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -249,15 +249,22 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } - def slaveLost(slaveId: String) { + def slaveLost(slaveId: String, reason: ExecutorLossReason) { var failedHost: Option[String] = None synchronized { val host = slaveIdToHost(slaveId) if (hostsAlive.contains(host)) { + logError("Lost an executor on " + host + ": " + reason) slaveIdsWithExecutors -= slaveId hostsAlive -= host activeTaskSetsQueue.foreach(_.hostLost(host)) failedHost = Some(host) + } else { + // We may get multiple slaveLost() calls with different loss reasons. For example, one + // may be triggered by a dropped connection from the slave while another may be a report + // of executor termination from Mesos. We produce log messages for both so we eventually + // report the termination reason. + logError("Lost an executor on " + host + " (already removed): " + reason) } } if (failedHost != None) { diff --git a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala new file mode 100644 index 0000000000000000000000000000000000000000..bba7de6a65c3d17aab47bdfa07c464ee7e801604 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala @@ -0,0 +1,21 @@ +package spark.scheduler.cluster + +import spark.executor.ExecutorExitCode + +/** + * Represents an explanation for a executor or whole slave failing or exiting. + */ +private[spark] +class ExecutorLossReason(val message: String) { + override def toString: String = message +} + +private[spark] +case class ExecutorExited(val exitCode: Int) + extends ExecutorLossReason(ExecutorExitCode.explainExitCode(exitCode)) { +} + +private[spark] +case class SlaveLost(_message: String = "Slave lost") + extends ExecutorLossReason(_message) { +} diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 7aba7324ab00992702a283caf164f0f631d06299..e2301347e510340b25c4d10731dc6db7987048c0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -19,6 +19,7 @@ private[spark] class SparkDeploySchedulerBackend( var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt + val executorIdToSlaveId = new HashMap[String, String] // Memory used by each executor (in megabytes) val executorMemory = { @@ -65,9 +66,23 @@ private[spark] class SparkDeploySchedulerBackend( } def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int) { + executorIdToSlaveId += id -> workerId logInfo("Granted executor ID %s on host %s with %d cores, %s RAM".format( id, host, cores, Utils.memoryMegabytesToString(memory))) } - def executorRemoved(id: String, message: String) {} + def executorRemoved(id: String, message: String, exitStatus: Option[Int]) { + val reason: ExecutorLossReason = exitStatus match { + case Some(code) => ExecutorExited(code) + case None => SlaveLost(message) + } + logInfo("Executor %s removed: %s".format(id, message)) + executorIdToSlaveId.get(id) match { + case Some(slaveId) => + executorIdToSlaveId.remove(id) + scheduler.slaveLost(slaveId, reason) + case None => + logInfo("No slave ID known for executor %s".format(id)) + } + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index d2cce0dc05511f8dbfc3c18b0b6618fae647baa7..eeaae23dc86978abb285ecb776aafa0e049baf8e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -69,13 +69,13 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor context.stop(self) case Terminated(actor) => - actorToSlaveId.get(actor).foreach(removeSlave) + actorToSlaveId.get(actor).foreach(removeSlave(_, "Akka actor terminated")) case RemoteClientDisconnected(transport, address) => - addressToSlaveId.get(address).foreach(removeSlave) + addressToSlaveId.get(address).foreach(removeSlave(_, "remote Akka client disconnected")) case RemoteClientShutdown(transport, address) => - addressToSlaveId.get(address).foreach(removeSlave) + addressToSlaveId.get(address).foreach(removeSlave(_, "remote Akka client shutdown")) } // Make fake resource offers on all slaves @@ -99,7 +99,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor } // Remove a disconnected slave from the cluster - def removeSlave(slaveId: String) { + def removeSlave(slaveId: String, reason: String) { logInfo("Slave " + slaveId + " disconnected, so removing it") val numCores = freeCores(slaveId) actorToSlaveId -= slaveActor(slaveId) @@ -109,7 +109,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor freeCores -= slaveId slaveHost -= slaveId totalCoreCount.addAndGet(-numCores) - scheduler.slaveLost(slaveId) + scheduler.slaveLost(slaveId, SlaveLost(reason)) } } diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index 814443fa52745a11fab771cf8de6a28bb90e792e..8c7a1dfbc0570b80af2517a7a572eef106c9badd 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -267,17 +267,23 @@ private[spark] class MesosSchedulerBackend( override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { + private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { logInfo("Mesos slave lost: " + slaveId.getValue) synchronized { slaveIdsWithExecutors -= slaveId.getValue } - scheduler.slaveLost(slaveId.getValue) + scheduler.slaveLost(slaveId.getValue, reason) + } + + override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { + recordSlaveLost(d, slaveId, SlaveLost()) } - override def executorLost(d: SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int) { - logInfo("Executor lost: %s, marking slave %s as lost".format(e.getValue, s.getValue)) - slaveLost(d, s) + override def executorLost(d: SchedulerDriver, executorId: ExecutorID, + slaveId: SlaveID, status: Int) { + logInfo("Executor lost: %s, marking slave %s as lost".format(executorId.getValue, + slaveId.getValue)) + recordSlaveLost(d, slaveId, ExecutorExited(status)) } // TODO: query Mesos for number of cores diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 1e36578e1a5c53f89b16f62ea8e33468453c344b..7a8ac10cdd88e51ebd06588ca6b7801cc9a693d1 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -1,22 +1,26 @@ package spark.storage +import java.io.{InputStream, OutputStream} +import java.nio.{ByteBuffer, MappedByteBuffer} +import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import scala.collection.JavaConversions._ + +import akka.actor.{ActorSystem, Cancellable, Props} import akka.dispatch.{Await, Future} import akka.util.Duration +import akka.util.duration._ -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - -import java.io.{InputStream, OutputStream, Externalizable, ObjectInput, ObjectOutput} -import java.nio.{MappedByteBuffer, ByteBuffer} -import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import spark.{CacheTracker, Logging, SizeEstimator, SparkException, Utils} +import spark.{CacheTracker, Logging, SizeEstimator, SparkEnv, SparkException, Utils} import spark.network._ import spark.serializer.Serializer -import spark.util.{MetadataCleaner, TimeStampedHashMap, ByteBufferInputStream} +import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} -import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import sun.nio.ch.DirectBuffer @@ -25,7 +29,11 @@ case class BlockException(blockId: String, message: String, ex: Exception = null extends Exception(message) private[spark] -class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, maxMemory: Long) +class BlockManager( + actorSystem: ActorSystem, + val master: BlockManagerMaster, + val serializer: Serializer, + maxMemory: Long) extends Logging { class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) { @@ -51,7 +59,7 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m } } - private val blockInfo = new TimeStampedHashMap[String, BlockInfo]() + private val blockInfo = new TimeStampedHashMap[String, BlockInfo] private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) private[storage] val diskStore: BlockStore = @@ -78,16 +86,31 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m // Whether to compress RDD partitions that are stored serialized val compressRdds = System.getProperty("spark.rdd.compress", "false").toBoolean + val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties + val host = System.getProperty("spark.hostname", Utils.localHostName()) + val slaveActor = master.actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), + name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) + + @volatile private var shuttingDown = false + + private def heartBeat() { + if (!master.sendHeartBeat(blockManagerId)) { + reregister() + } + } + + var heartBeatTask: Cancellable = null + val metadataCleaner = new MetadataCleaner("BlockManager", this.dropOldBlocks) initialize() /** * Construct a BlockManager with a memory limit set based on system properties. */ - def this(master: BlockManagerMaster, serializer: Serializer) = { - this(master, serializer, BlockManager.getMaxMemoryFromSystemProperties) + def this(actorSystem: ActorSystem, master: BlockManagerMaster, serializer: Serializer) = { + this(actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) } /** @@ -95,46 +118,94 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m * BlockManagerWorker actor. */ private def initialize() { - master.registerBlockManager(blockManagerId, maxMemory) + master.registerBlockManager(blockManagerId, maxMemory, slaveActor) BlockManagerWorker.startBlockManagerWorker(this) + if (!BlockManager.getDisableHeartBeatsForTesting) { + heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) { + heartBeat() + } + } } /** - * Get storage level of local block. If no info exists for the block, then returns null. + * Report all blocks to the BlockManager again. This may be necessary if we are dropped + * by the BlockManager and come back or if we become capable of recovering blocks on disk after + * an executor crash. + * + * This function deliberately fails silently if the master returns false (indicating that + * the slave needs to reregister). The error condition will be detected again by the next + * heart beat attempt or new block registration and another try to reregister all blocks + * will be made then. */ - def getLevel(blockId: String): StorageLevel = { - blockInfo.get(blockId).map(_.level).orNull + private def reportAllBlocks() { + logInfo("Reporting " + blockInfo.size + " blocks to the master.") + for ((blockId, info) <- blockInfo) { + if (!tryToReportBlockStatus(blockId, info)) { + logError("Failed to report " + blockId + " to master; giving up.") + return + } + } } /** - * Tell the master about the current storage status of a block. This will send a heartbeat + * Reregister with the master and report all blocks to it. This will be called by the heart beat + * thread if our heartbeat to the block amnager indicates that we were not registered. + */ + def reregister() { + // TODO: We might need to rate limit reregistering. + logInfo("BlockManager reregistering with master") + master.registerBlockManager(blockManagerId, maxMemory, slaveActor) + reportAllBlocks() + } + + /** + * Get storage level of local block. If no info exists for the block, then returns null. + */ + def getLevel(blockId: String): StorageLevel = blockInfo.get(blockId).map(_.level).orNull + + /** + * Tell the master about the current storage status of a block. This will send a block update * message reflecting the current status, *not* the desired storage level in its block info. * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk. */ - def reportBlockStatus(blockId: String) { - val (curLevel, inMemSize, onDiskSize) = blockInfo.get(blockId) match { - case None => - (StorageLevel.NONE, 0L, 0L) - case Some(info) => - info.synchronized { - info.level match { - case null => - (StorageLevel.NONE, 0L, 0L) - case level => - val inMem = level.useMemory && memoryStore.contains(blockId) - val onDisk = level.useDisk && diskStore.contains(blockId) - ( - new StorageLevel(onDisk, inMem, level.deserialized, level.replication), - if (inMem) memoryStore.getSize(blockId) else 0L, - if (onDisk) diskStore.getSize(blockId) else 0L - ) - } - } + def reportBlockStatus(blockId: String, info: BlockInfo) { + val needReregister = !tryToReportBlockStatus(blockId, info) + if (needReregister) { + logInfo("Got told to reregister updating block " + blockId) + // Reregistering will report our new block for free. + reregister() } - master.updateBlockInfo(blockManagerId, blockId, curLevel, inMemSize, onDiskSize) logDebug("Told master about block " + blockId) } + /** + * Actually send a UpdateBlockInfo message. Returns the mater's response, + * which will be true if the block was successfully recorded and false if + * the slave needs to re-register. + */ + private def tryToReportBlockStatus(blockId: String, info: BlockInfo): Boolean = { + val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized { + info.level match { + case null => + (StorageLevel.NONE, 0L, 0L, false) + case level => + val inMem = level.useMemory && memoryStore.contains(blockId) + val onDisk = level.useDisk && diskStore.contains(blockId) + val storageLevel = new StorageLevel(onDisk, inMem, level.deserialized, level.replication) + val memSize = if (inMem) memoryStore.getSize(blockId) else 0L + val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L + (storageLevel, memSize, diskSize, info.tellMaster) + } + } + + if (tellMaster) { + master.updateBlockInfo(blockManagerId, blockId, curLevel, inMemSize, onDiskSize) + } else { + true + } + } + + /** * Get locations of the block. */ @@ -302,7 +373,6 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m } else { logDebug("Block " + blockId + " not registered locally") } - return None } @@ -572,7 +642,7 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m // and tell the master about it. myInfo.markReady(size) if (tellMaster) { - reportBlockStatus(blockId) + reportBlockStatus(blockId, myInfo) } } logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) @@ -659,7 +729,7 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m // and tell the master about it. myInfo.markReady(bytes.limit) if (tellMaster) { - reportBlockStatus(blockId) + reportBlockStatus(blockId, myInfo) } } @@ -741,7 +811,7 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m def dropFromMemory(blockId: String, data: Either[ArrayBuffer[Any], ByteBuffer]) { logInfo("Dropping block " + blockId + " from memory") val info = blockInfo.get(blockId).orNull - if (info != null) { + if (info != null) { info.synchronized { val level = info.level if (level.useDisk && !diskStore.contains(blockId)) { @@ -753,9 +823,12 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m diskStore.putBytes(blockId, bytes, level) } } - memoryStore.remove(blockId) + val blockWasRemoved = memoryStore.remove(blockId) + if (!blockWasRemoved) { + logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") + } if (info.tellMaster) { - reportBlockStatus(blockId) + reportBlockStatus(blockId, info) } if (!level.useDisk) { // The block is completely gone from this node; forget it so we can put() it again later. @@ -767,10 +840,34 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m } } + /** + * Remove a block from both memory and disk. + */ + def removeBlock(blockId: String) { + logInfo("Removing block " + blockId) + val info = blockInfo.get(blockId).orNull + if (info != null) info.synchronized { + // Removals are idempotent in disk store and memory store. At worst, we get a warning. + val removedFromMemory = memoryStore.remove(blockId) + val removedFromDisk = diskStore.remove(blockId) + if (!removedFromMemory && !removedFromDisk) { + logWarning("Block " + blockId + " could not be removed as it was not found in either " + + "the disk or memory store") + } + blockInfo.remove(blockId) + if (info.tellMaster) { + reportBlockStatus(blockId, info) + } + } else { + // The block has already been removed; do nothing. + logWarning("Asked to remove block " + blockId + ", which does not exist") + } + } + def dropOldBlocks(cleanupTime: Long) { logInfo("Dropping blocks older than " + cleanupTime) val iterator = blockInfo.internalMap.entrySet().iterator() - while(iterator.hasNext) { + while (iterator.hasNext) { val entry = iterator.next() val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2) if (time < cleanupTime) { @@ -785,7 +882,7 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m iterator.remove() logInfo("Dropped block " + id) } - reportBlockStatus(id) + reportBlockStatus(id, info) } } } @@ -835,7 +932,11 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m } def stop() { + if (heartBeatTask != null) { + heartBeatTask.cancel() + } connectionManager.stop() + master.actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diskStore.clear() @@ -845,11 +946,20 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m private[spark] object BlockManager extends Logging { + + val ID_GENERATOR = new IdGenerator + def getMaxMemoryFromSystemProperties: Long = { val memoryFraction = System.getProperty("spark.storage.memoryFraction", "0.66").toDouble (Runtime.getRuntime.maxMemory * memoryFraction).toLong } + def getHeartBeatFrequencyFromSystemProperties: Long = + System.getProperty("spark.storage.blockManagerHeartBeatMs", "5000").toLong + + def getDisableHeartBeatsForTesting: Boolean = + System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean + /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that * might cause errors if one attempts to read from the unmapped buffer, but it's better than diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index 4933cc660623a41037297b4a66448fa0b72332b6..488679f0496b42d4cbb59ba6205583a4992584b0 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -1,8 +1,9 @@ package spark.storage -import java.io.{IOException, ObjectOutput, ObjectInput, Externalizable} +import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap + private[spark] class BlockManagerId(var ip: String, var port: Int) extends Externalizable { def this() = this(null, 0) // For deserialization only @@ -19,10 +20,7 @@ private[spark] class BlockManagerId(var ip: String, var port: Int) extends Exter } @throws(classOf[IOException]) - private def readResolve(): Object = { - BlockManagerId.getCachedBlockManagerId(this) - } - + private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this) override def toString = "BlockManagerId(" + ip + ", " + port + ")" @@ -34,7 +32,9 @@ private[spark] class BlockManagerId(var ip: String, var port: Int) extends Exter } } -object BlockManagerId { + +private[spark] object BlockManagerId { + val blockManagerIdCache = new ConcurrentHashMap[BlockManagerId, BlockManagerId]() def getCachedBlockManagerId(id: BlockManagerId): BlockManagerId = { @@ -45,4 +45,4 @@ object BlockManagerId { id } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index af156636213f91ada9195cc4c76d27a448c7cbd7..a3d8671834dbc676c83e4a739c40462127a93e93 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -1,384 +1,140 @@ package spark.storage -import java.io._ -import java.util.{HashMap => JHashMap} - -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.ArrayBuffer import scala.util.Random -import akka.actor._ -import akka.dispatch._ +import akka.actor.{Actor, ActorRef, ActorSystem, Props} +import akka.dispatch.Await import akka.pattern.ask -import akka.remote._ import akka.util.{Duration, Timeout} import akka.util.duration._ import spark.{Logging, SparkException, Utils} -private[spark] -sealed trait ToBlockManagerMaster - -private[spark] -case class RegisterBlockManager( - blockManagerId: BlockManagerId, - maxMemSize: Long) - extends ToBlockManagerMaster - -private[spark] -class UpdateBlockInfo( - var blockManagerId: BlockManagerId, - var blockId: String, - var storageLevel: StorageLevel, - var memSize: Long, - var diskSize: Long) - extends ToBlockManagerMaster - with Externalizable { - - def this() = this(null, null, null, 0, 0) // For deserialization only - - override def writeExternal(out: ObjectOutput) { - blockManagerId.writeExternal(out) - out.writeUTF(blockId) - storageLevel.writeExternal(out) - out.writeInt(memSize.toInt) - out.writeInt(diskSize.toInt) - } - - override def readExternal(in: ObjectInput) { - blockManagerId = new BlockManagerId() - blockManagerId.readExternal(in) - blockId = in.readUTF() - storageLevel = new StorageLevel() - storageLevel.readExternal(in) - memSize = in.readInt() - diskSize = in.readInt() - } -} - -private[spark] -object UpdateBlockInfo { - def apply(blockManagerId: BlockManagerId, - blockId: String, - storageLevel: StorageLevel, - memSize: Long, - diskSize: Long): UpdateBlockInfo = { - new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) - } - - // For pattern-matching - def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) - } -} - -private[spark] -case class GetLocations(blockId: String) extends ToBlockManagerMaster - -private[spark] -case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster - -private[spark] -case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster - -private[spark] -case class RemoveHost(host: String) extends ToBlockManagerMaster - -private[spark] -case object StopBlockManagerMaster extends ToBlockManagerMaster - -private[spark] -case object GetMemoryStatus extends ToBlockManagerMaster - - -private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { - - class BlockManagerInfo( - val blockManagerId: BlockManagerId, - timeMs: Long, - val maxMem: Long) { - private var _lastSeenMs = timeMs - private var _remainingMem = maxMem - private val _blocks = new JHashMap[String, StorageLevel] - - logInfo("Registering block manager %s:%d with %s RAM".format( - blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(maxMem))) - - def updateLastSeenMs() { - _lastSeenMs = System.currentTimeMillis() / 1000 - } - - def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long, diskSize: Long) - : Unit = synchronized { - - updateLastSeenMs() - - if (_blocks.containsKey(blockId)) { - // The block exists on the slave already. - val originalLevel: StorageLevel = _blocks.get(blockId) - - if (originalLevel.useMemory) { - _remainingMem += memSize - } - } - - if (storageLevel.isValid) { - // isValid means it is either stored in-memory or on-disk. - _blocks.put(blockId, storageLevel) - if (storageLevel.useMemory) { - _remainingMem -= memSize - logInfo("Added %s in memory on %s:%d (size: %s, free: %s)".format( - blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize), - Utils.memoryBytesToString(_remainingMem))) - } - if (storageLevel.useDisk) { - logInfo("Added %s on disk on %s:%d (size: %s)".format( - blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize))) - } - } else if (_blocks.containsKey(blockId)) { - // If isValid is not true, drop the block. - val originalLevel: StorageLevel = _blocks.get(blockId) - _blocks.remove(blockId) - if (originalLevel.useMemory) { - _remainingMem += memSize - logInfo("Removed %s on %s:%d in memory (size: %s, free: %s)".format( - blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize), - Utils.memoryBytesToString(_remainingMem))) - } - if (originalLevel.useDisk) { - logInfo("Removed %s on %s:%d on disk (size: %s)".format( - blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize))) - } - } - } - - def remainingMem: Long = _remainingMem +private[spark] class BlockManagerMaster( + val actorSystem: ActorSystem, + isMaster: Boolean, + isLocal: Boolean, + masterIp: String, + masterPort: Int) + extends Logging { - def lastSeenMs: Long = _lastSeenMs + val AKKA_RETRY_ATTEMPS: Int = System.getProperty("spark.akka.num.retries", "3").toInt + val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt - override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem + val MASTER_AKKA_ACTOR_NAME = "BlockMasterManager" + val SLAVE_AKKA_ACTOR_NAME = "BlockSlaveManager" + val DEFAULT_MANAGER_IP: String = Utils.localHostName() - def clear() { - _blocks.clear() + val timeout = 10.seconds + var masterActor: ActorRef = { + if (isMaster) { + val masterActor = actorSystem.actorOf(Props(new BlockManagerMasterActor(isLocal)), + name = MASTER_AKKA_ACTOR_NAME) + logInfo("Registered BlockManagerMaster Actor") + masterActor + } else { + val url = "akka://spark@%s:%s/user/%s".format(masterIp, masterPort, MASTER_AKKA_ACTOR_NAME) + logInfo("Connecting to BlockManagerMaster: " + url) + actorSystem.actorFor(url) } } - private val blockManagerInfo = new HashMap[BlockManagerId, BlockManagerInfo] - private val blockInfo = new JHashMap[String, Pair[Int, HashSet[BlockManagerId]]] - - initLogging() - - def removeHost(host: String) { - logInfo("Trying to remove the host: " + host + " from BlockManagerMaster.") - logInfo("Previous hosts: " + blockManagerInfo.keySet.toSeq) - val ip = host.split(":")(0) - val port = host.split(":")(1) - blockManagerInfo.remove(new BlockManagerId(ip, port.toInt)) - logInfo("Current hosts: " + blockManagerInfo.keySet.toSeq) - sender ! true - } - - def receive = { - case RegisterBlockManager(blockManagerId, maxMemSize) => - register(blockManagerId, maxMemSize) - - case UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => - updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) - - case GetLocations(blockId) => - getLocations(blockId) - - case GetLocationsMultipleBlockIds(blockIds) => - getLocationsMultipleBlockIds(blockIds) - - case GetPeers(blockManagerId, size) => - getPeersDeterministic(blockManagerId, size) - /*getPeers(blockManagerId, size)*/ - - case GetMemoryStatus => - getMemoryStatus - - case RemoveHost(host) => - removeHost(host) - sender ! true - - case StopBlockManagerMaster => - logInfo("Stopping BlockManagerMaster") - sender ! true - context.stop(self) - - case other => - logInfo("Got unknown message: " + other) + /** Remove a dead host from the master actor. This is only called on the master side. */ + def notifyADeadHost(host: String) { + tell(RemoveHost(host)) + logInfo("Removed " + host + " successfully in notifyADeadHost") } - // Return a map from the block manager id to max memory and remaining memory. - private def getMemoryStatus() { - val res = blockManagerInfo.map { case(blockManagerId, info) => - (blockManagerId, (info.maxMem, info.remainingMem)) - }.toMap - sender ! res + /** + * Send the master actor a heart beat from the slave. Returns true if everything works out, + * false if the master does not know about the given block manager, which means the block + * manager should re-register. + */ + def sendHeartBeat(blockManagerId: BlockManagerId): Boolean = { + askMasterWithRetry[Boolean](HeartBeat(blockManagerId)) } - private def register(blockManagerId: BlockManagerId, maxMemSize: Long) { - val startTimeMs = System.currentTimeMillis() - val tmp = " " + blockManagerId + " " - logDebug("Got in register 0" + tmp + Utils.getUsedTimeMs(startTimeMs)) - if (blockManagerId.ip == Utils.localHostName() && !isLocal) { - logInfo("Got Register Msg from master node, don't register it") - } else { - blockManagerInfo += (blockManagerId -> new BlockManagerInfo( - blockManagerId, System.currentTimeMillis() / 1000, maxMemSize)) - } - logDebug("Got in register 1" + tmp + Utils.getUsedTimeMs(startTimeMs)) - sender ! true + /** Register the BlockManager's id with the master. */ + def registerBlockManager( + blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + logInfo("Trying to register BlockManager") + tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveActor)) + logInfo("Registered BlockManager") } - private def updateBlockInfo( + def updateBlockInfo( blockManagerId: BlockManagerId, blockId: String, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { - - val startTimeMs = System.currentTimeMillis() - val tmp = " " + blockManagerId + " " + blockId + " " - - if (!blockManagerInfo.contains(blockManagerId)) { - // Can happen if this is from a locally cached partition on the master - sender ! true - return - } - - if (blockId == null) { - blockManagerInfo(blockManagerId).updateLastSeenMs() - logDebug("Got in updateBlockInfo 1" + tmp + " used " + Utils.getUsedTimeMs(startTimeMs)) - sender ! true - } - - blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize) - - var locations: HashSet[BlockManagerId] = null - if (blockInfo.containsKey(blockId)) { - locations = blockInfo.get(blockId)._2 - } else { - locations = new HashSet[BlockManagerId] - blockInfo.put(blockId, (storageLevel.replication, locations)) - } - - if (storageLevel.isValid) { - locations += blockManagerId - } else { - locations.remove(blockManagerId) - } - - if (locations.size == 0) { - blockInfo.remove(blockId) - } - sender ! true + diskSize: Long): Boolean = { + val res = askMasterWithRetry[Boolean]( + UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize)) + logInfo("Updated info of block " + blockId) + res } - private def getLocations(blockId: String) { - val startTimeMs = System.currentTimeMillis() - val tmp = " " + blockId + " " - logDebug("Got in getLocations 0" + tmp + Utils.getUsedTimeMs(startTimeMs)) - if (blockInfo.containsKey(blockId)) { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - res.appendAll(blockInfo.get(blockId)._2) - logDebug("Got in getLocations 1" + tmp + " as "+ res.toSeq + " at " - + Utils.getUsedTimeMs(startTimeMs)) - sender ! res.toSeq - } else { - logDebug("Got in getLocations 2" + tmp + Utils.getUsedTimeMs(startTimeMs)) - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - sender ! res - } + /** Get locations of the blockId from the master */ + def getLocations(blockId: String): Seq[BlockManagerId] = { + askMasterWithRetry[Seq[BlockManagerId]](GetLocations(blockId)) } - private def getLocationsMultipleBlockIds(blockIds: Array[String]) { - def getLocations(blockId: String): Seq[BlockManagerId] = { - val tmp = blockId - logDebug("Got in getLocationsMultipleBlockIds Sub 0 " + tmp) - if (blockInfo.containsKey(blockId)) { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - res.appendAll(blockInfo.get(blockId)._2) - logDebug("Got in getLocationsMultipleBlockIds Sub 1 " + tmp + " " + res.toSeq) - return res.toSeq - } else { - logDebug("Got in getLocationsMultipleBlockIds Sub 2 " + tmp) - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - return res.toSeq - } - } - - logDebug("Got in getLocationsMultipleBlockIds " + blockIds.toSeq) - var res: ArrayBuffer[Seq[BlockManagerId]] = new ArrayBuffer[Seq[BlockManagerId]] - for (blockId <- blockIds) { - res.append(getLocations(blockId)) - } - logDebug("Got in getLocationsMultipleBlockIds " + blockIds.toSeq + " : " + res.toSeq) - sender ! res.toSeq + /** Get locations of multiple blockIds from the master */ + def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = { + askMasterWithRetry[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) } - private def getPeers(blockManagerId: BlockManagerId, size: Int) { - var peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - res.appendAll(peers) - res -= blockManagerId - val rand = new Random(System.currentTimeMillis()) - while (res.length > size) { - res.remove(rand.nextInt(res.length)) + /** Get ids of other nodes in the cluster from the master */ + def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { + val result = askMasterWithRetry[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) + if (result.length != numPeers) { + throw new SparkException( + "Error getting peers, only got " + result.size + " instead of " + numPeers) } - sender ! res.toSeq + result } - private def getPeersDeterministic(blockManagerId: BlockManagerId, size: Int) { - var peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] + /** + * Remove a block from the slaves that have it. This can only be used to remove + * blocks that the master knows about. + */ + def removeBlock(blockId: String) { + askMasterWithRetry(RemoveBlock(blockId)) + } - val peersWithIndices = peers.zipWithIndex - val selfIndex = peersWithIndices.find(_._1 == blockManagerId).map(_._2).getOrElse(-1) - if (selfIndex == -1) { - throw new Exception("Self index for " + blockManagerId + " not found") - } + /** + * Return the memory status for each block manager, in the form of a map from + * the block manager's id to two long values. The first value is the maximum + * amount of memory allocated for the block manager, while the second is the + * amount of remaining memory. + */ + def getMemoryStatus: Map[BlockManagerId, (Long, Long)] = { + askMasterWithRetry[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) + } - // Note that this logic will select the same node multiple times if there aren't enough peers - var index = selfIndex - while (res.size < size) { - index += 1 - if (index == selfIndex) { - throw new Exception("More peer expected than available") - } - res += peers(index % peers.size) + /** Stop the master actor, called only on the Spark master node */ + def stop() { + if (masterActor != null) { + tell(StopBlockManagerMaster) + masterActor = null + logInfo("BlockManagerMaster stopped") } - sender ! res.toSeq } -} -private[spark] class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: Boolean) - extends Logging { - - val actorName = "BlockMasterManager" - val timeout = 10.seconds - val maxAttempts = 5 - - var masterActor = if (isMaster) { - val actor = actorSystem.actorOf(Props(new BlockManagerMasterActor(isLocal)), name = actorName) - logInfo("Registered BlockManagerMaster Actor") - actor - } else { - val host = System.getProperty("spark.master.host", "localhost") - val port = System.getProperty("spark.master.port", "7077").toInt - val url = "akka://spark@%s:%s/user/%s".format(host, port, actorName) - val actor = actorSystem.actorFor(url) - logInfo("Connecting to BlockManagerMaster: " + url) - actor + /** Send a one-way message to the master actor, to which we expect it to reply with true. */ + private def tell(message: Any) { + if (!askMasterWithRetry[Boolean](message)) { + throw new SparkException("BlockManagerMasterActor returned false, expected true.") + } } /** * Send a message to the master actor and get its result within a default timeout, or * throw a SparkException if this fails. */ - private def ask[T](message: Any): T = { + private def askMasterWithRetry[T](message: Any): T = { // TODO: Consider removing multiple attempts if (masterActor == null) { throw new SparkException("Error sending message to BlockManager as masterActor is null " + @@ -386,9 +142,9 @@ private[spark] class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Bool } var attempts = 0 var lastException: Exception = null - while (attempts < maxAttempts) { + while (attempts < AKKA_RETRY_ATTEMPS) { attempts += 1 - try { + try { val future = masterActor.ask(message)(timeout) val result = Await.result(future, timeout) if (result == null) { @@ -396,85 +152,16 @@ private[spark] class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Bool } return result.asInstanceOf[T] } catch { - case ie: InterruptedException => - throw ie + case ie: InterruptedException => throw ie case e: Exception => lastException = e - logWarning( - "Error sending message to BlockManagerMaster in " + attempts + " attempts", e) + logWarning("Error sending message to BlockManagerMaster in " + attempts + " attempts", e) } - Thread.sleep(100) + Thread.sleep(AKKA_RETRY_INTERVAL_MS) } + throw new SparkException( "Error sending message to BlockManagerMaster [message = " + message + "]", lastException) } - /** - * Send a one-way message to the master actor, to which we expect it to reply with true - */ - private def tell(message: Any) { - if (!ask[Boolean](message)) { - throw new SparkException("Telling master a message returned false") - } - } - - /** - * Register the BlockManager's id with the master - */ - def registerBlockManager(blockManagerId: BlockManagerId, maxMemSize: Long) { - logInfo("Trying to register BlockManager") - tell(RegisterBlockManager(blockManagerId, maxMemSize)) - logInfo("Registered BlockManager") - } - - def updateBlockInfo( - blockManagerId: BlockManagerId, - blockId: String, - storageLevel: StorageLevel, - memSize: Long, - diskSize: Long - ) { - tell(UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize)) - logInfo("Updated info of block " + blockId) - } - - /** Get locations of the blockId from the master */ - def getLocations(blockId: String): Seq[BlockManagerId] = { - ask[Seq[BlockManagerId]](GetLocations(blockId)) - } - - /** Get locations of multiple blockIds from the master */ - def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = { - ask[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) - } - - /** Get ids of other nodes in the cluster from the master */ - def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { - val result = ask[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) - if (result.length != numPeers) { - throw new SparkException( - "Error getting peers, only got " + result.size + " instead of " + numPeers) - } - result - } - - /** Notify the master of a dead node */ - def notifyADeadHost(host: String) { - tell(RemoveHost(host + ":10902")) - logInfo("Told BlockManagerMaster to remove dead host " + host) - } - - /** Get the memory status form the master */ - def getMemoryStatus(): Map[BlockManagerId, (Long, Long)] = { - ask[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) - } - - /** Stop the master actor, called only on the Spark master node */ - def stop() { - if (masterActor != null) { - tell(StopBlockManagerMaster) - masterActor = null - logInfo("BlockManagerMaster stopped") - } - } } diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala new file mode 100644 index 0000000000000000000000000000000000000000..f4d026da3329c801775275899a7f2dd94136f6dc --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -0,0 +1,401 @@ +package spark.storage + +import java.util.{HashMap => JHashMap} + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.JavaConversions._ +import scala.util.Random + +import akka.actor.{Actor, ActorRef, Cancellable} +import akka.util.{Duration, Timeout} +import akka.util.duration._ + +import spark.{Logging, Utils} + +/** + * BlockManagerMasterActor is an actor on the master node to track statuses of + * all slaves' block managers. + */ +private[spark] +class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { + + // Mapping from block manager id to the block manager's information. + private val blockManagerInfo = + new HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo] + + // Mapping from host name to block manager id. We allow multiple block managers + // on the same host name (ip). + private val blockManagerIdByHost = new HashMap[String, ArrayBuffer[BlockManagerId]] + + // Mapping from block id to the set of block managers that have the block. + private val blockLocations = new JHashMap[String, Pair[Int, HashSet[BlockManagerId]]] + + initLogging() + + val slaveTimeout = System.getProperty("spark.storage.blockManagerSlaveTimeoutMs", + "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong + + val checkTimeoutInterval = System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", + "5000").toLong + + var timeoutCheckingTask: Cancellable = null + + override def preStart() { + if (!BlockManager.getDisableHeartBeatsForTesting) { + timeoutCheckingTask = context.system.scheduler.schedule( + 0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) + } + super.preStart() + } + + def receive = { + case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => + register(blockManagerId, maxMemSize, slaveActor) + + case UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => + updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) + + case GetLocations(blockId) => + getLocations(blockId) + + case GetLocationsMultipleBlockIds(blockIds) => + getLocationsMultipleBlockIds(blockIds) + + case GetPeers(blockManagerId, size) => + getPeersDeterministic(blockManagerId, size) + /*getPeers(blockManagerId, size)*/ + + case GetMemoryStatus => + getMemoryStatus + + case RemoveBlock(blockId) => + removeBlock(blockId) + + case RemoveHost(host) => + removeHost(host) + sender ! true + + case StopBlockManagerMaster => + logInfo("Stopping BlockManagerMaster") + sender ! true + if (timeoutCheckingTask != null) { + timeoutCheckingTask.cancel + } + context.stop(self) + + case ExpireDeadHosts => + expireDeadHosts() + + case HeartBeat(blockManagerId) => + heartBeat(blockManagerId) + + case other => + logInfo("Got unknown message: " + other) + } + + def removeBlockManager(blockManagerId: BlockManagerId) { + val info = blockManagerInfo(blockManagerId) + + // Remove the block manager from blockManagerIdByHost. If the list of block + // managers belonging to the IP is empty, remove the entry from the hash map. + blockManagerIdByHost.get(blockManagerId.ip).foreach { managers: ArrayBuffer[BlockManagerId] => + managers -= blockManagerId + if (managers.size == 0) blockManagerIdByHost.remove(blockManagerId.ip) + } + + // Remove it from blockManagerInfo and remove all the blocks. + blockManagerInfo.remove(blockManagerId) + var iterator = info.blocks.keySet.iterator + while (iterator.hasNext) { + val blockId = iterator.next + val locations = blockLocations.get(blockId)._2 + locations -= blockManagerId + if (locations.size == 0) { + blockLocations.remove(locations) + } + } + } + + def expireDeadHosts() { + logDebug("Checking for hosts with no recent heart beats in BlockManagerMaster.") + val now = System.currentTimeMillis() + val minSeenTime = now - slaveTimeout + val toRemove = new HashSet[BlockManagerId] + for (info <- blockManagerInfo.values) { + if (info.lastSeenMs < minSeenTime) { + logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats") + toRemove += info.blockManagerId + } + } + toRemove.foreach(removeBlockManager) + } + + def removeHost(host: String) { + logInfo("Trying to remove the host: " + host + " from BlockManagerMaster.") + logInfo("Previous hosts: " + blockManagerInfo.keySet.toSeq) + blockManagerIdByHost.get(host).foreach(_.foreach(removeBlockManager)) + logInfo("Current hosts: " + blockManagerInfo.keySet.toSeq) + sender ! true + } + + def heartBeat(blockManagerId: BlockManagerId) { + if (!blockManagerInfo.contains(blockManagerId)) { + if (blockManagerId.ip == Utils.localHostName() && !isLocal) { + sender ! true + } else { + sender ! false + } + } else { + blockManagerInfo(blockManagerId).updateLastSeenMs() + sender ! true + } + } + + // Remove a block from the slaves that have it. This can only be used to remove + // blocks that the master knows about. + private def removeBlock(blockId: String) { + val block = blockLocations.get(blockId) + if (block != null) { + block._2.foreach { blockManagerId: BlockManagerId => + val blockManager = blockManagerInfo.get(blockManagerId) + if (blockManager.isDefined) { + // Remove the block from the slave's BlockManager. + // Doesn't actually wait for a confirmation and the message might get lost. + // If message loss becomes frequent, we should add retry logic here. + blockManager.get.slaveActor ! RemoveBlock(blockId) + } + } + } + sender ! true + } + + // Return a map from the block manager id to max memory and remaining memory. + private def getMemoryStatus() { + val res = blockManagerInfo.map { case(blockManagerId, info) => + (blockManagerId, (info.maxMem, info.remainingMem)) + }.toMap + sender ! res + } + + private def register(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + val startTimeMs = System.currentTimeMillis() + val tmp = " " + blockManagerId + " " + + if (blockManagerId.ip == Utils.localHostName() && !isLocal) { + logInfo("Got Register Msg from master node, don't register it") + } else { + blockManagerIdByHost.get(blockManagerId.ip) match { + case Some(managers) => + // A block manager of the same host name already exists. + logInfo("Got another registration for host " + blockManagerId) + managers += blockManagerId + case None => + blockManagerIdByHost += (blockManagerId.ip -> ArrayBuffer(blockManagerId)) + } + + blockManagerInfo += (blockManagerId -> new BlockManagerMasterActor.BlockManagerInfo( + blockManagerId, System.currentTimeMillis(), maxMemSize, slaveActor)) + } + sender ! true + } + + private def updateBlockInfo( + blockManagerId: BlockManagerId, + blockId: String, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long) { + + val startTimeMs = System.currentTimeMillis() + val tmp = " " + blockManagerId + " " + blockId + " " + + if (!blockManagerInfo.contains(blockManagerId)) { + if (blockManagerId.ip == Utils.localHostName() && !isLocal) { + // We intentionally do not register the master (except in local mode), + // so we should not indicate failure. + sender ! true + } else { + sender ! false + } + return + } + + if (blockId == null) { + blockManagerInfo(blockManagerId).updateLastSeenMs() + sender ! true + return + } + + blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize) + + var locations: HashSet[BlockManagerId] = null + if (blockLocations.containsKey(blockId)) { + locations = blockLocations.get(blockId)._2 + } else { + locations = new HashSet[BlockManagerId] + blockLocations.put(blockId, (storageLevel.replication, locations)) + } + + if (storageLevel.isValid) { + locations.add(blockManagerId) + } else { + locations.remove(blockManagerId) + } + + // Remove the block from master tracking if it has been removed on all slaves. + if (locations.size == 0) { + blockLocations.remove(blockId) + } + sender ! true + } + + private def getLocations(blockId: String) { + val startTimeMs = System.currentTimeMillis() + val tmp = " " + blockId + " " + if (blockLocations.containsKey(blockId)) { + var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] + res.appendAll(blockLocations.get(blockId)._2) + sender ! res.toSeq + } else { + var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] + sender ! res + } + } + + private def getLocationsMultipleBlockIds(blockIds: Array[String]) { + def getLocations(blockId: String): Seq[BlockManagerId] = { + val tmp = blockId + if (blockLocations.containsKey(blockId)) { + var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] + res.appendAll(blockLocations.get(blockId)._2) + return res.toSeq + } else { + var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] + return res.toSeq + } + } + + var res: ArrayBuffer[Seq[BlockManagerId]] = new ArrayBuffer[Seq[BlockManagerId]] + for (blockId <- blockIds) { + res.append(getLocations(blockId)) + } + sender ! res.toSeq + } + + private def getPeers(blockManagerId: BlockManagerId, size: Int) { + var peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray + var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] + res.appendAll(peers) + res -= blockManagerId + val rand = new Random(System.currentTimeMillis()) + while (res.length > size) { + res.remove(rand.nextInt(res.length)) + } + sender ! res.toSeq + } + + private def getPeersDeterministic(blockManagerId: BlockManagerId, size: Int) { + var peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray + var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] + + val selfIndex = peers.indexOf(blockManagerId) + if (selfIndex == -1) { + throw new Exception("Self index for " + blockManagerId + " not found") + } + + // Note that this logic will select the same node multiple times if there aren't enough peers + var index = selfIndex + while (res.size < size) { + index += 1 + if (index == selfIndex) { + throw new Exception("More peer expected than available") + } + res += peers(index % peers.size) + } + sender ! res.toSeq + } +} + + +private[spark] +object BlockManagerMasterActor { + + case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) + + class BlockManagerInfo( + val blockManagerId: BlockManagerId, + timeMs: Long, + val maxMem: Long, + val slaveActor: ActorRef) + extends Logging { + + private var _lastSeenMs: Long = timeMs + private var _remainingMem: Long = maxMem + + // Mapping from block id to its status. + private val _blocks = new JHashMap[String, BlockStatus] + + logInfo("Registering block manager %s:%d with %s RAM".format( + blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(maxMem))) + + def updateLastSeenMs() { + _lastSeenMs = System.currentTimeMillis() + } + + def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long, diskSize: Long) + : Unit = synchronized { + + updateLastSeenMs() + + if (_blocks.containsKey(blockId)) { + // The block exists on the slave already. + val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel + + if (originalLevel.useMemory) { + _remainingMem += memSize + } + } + + if (storageLevel.isValid) { + // isValid means it is either stored in-memory or on-disk. + _blocks.put(blockId, BlockStatus(storageLevel, memSize, diskSize)) + if (storageLevel.useMemory) { + _remainingMem -= memSize + logInfo("Added %s in memory on %s:%d (size: %s, free: %s)".format( + blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize), + Utils.memoryBytesToString(_remainingMem))) + } + if (storageLevel.useDisk) { + logInfo("Added %s on disk on %s:%d (size: %s)".format( + blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize))) + } + } else if (_blocks.containsKey(blockId)) { + // If isValid is not true, drop the block. + val blockStatus: BlockStatus = _blocks.get(blockId) + _blocks.remove(blockId) + if (blockStatus.storageLevel.useMemory) { + _remainingMem += blockStatus.memSize + logInfo("Removed %s on %s:%d in memory (size: %s, free: %s)".format( + blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize), + Utils.memoryBytesToString(_remainingMem))) + } + if (blockStatus.storageLevel.useDisk) { + logInfo("Removed %s on %s:%d on disk (size: %s)".format( + blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize))) + } + } + } + + def remainingMem: Long = _remainingMem + + def lastSeenMs: Long = _lastSeenMs + + def blocks: JHashMap[String, BlockStatus] = _blocks + + override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem + + def clear() { + _blocks.clear() + } + } +} diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala new file mode 100644 index 0000000000000000000000000000000000000000..d73a9b790f4793596ed9fce418af83fc62a7edf5 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -0,0 +1,102 @@ +package spark.storage + +import java.io.{Externalizable, ObjectInput, ObjectOutput} + +import akka.actor.ActorRef + + +////////////////////////////////////////////////////////////////////////////////// +// Messages from the master to slaves. +////////////////////////////////////////////////////////////////////////////////// +private[spark] +sealed trait ToBlockManagerSlave + +// Remove a block from the slaves that have it. This can only be used to remove +// blocks that the master knows about. +private[spark] +case class RemoveBlock(blockId: String) extends ToBlockManagerSlave + + +////////////////////////////////////////////////////////////////////////////////// +// Messages from slaves to the master. +////////////////////////////////////////////////////////////////////////////////// +private[spark] +sealed trait ToBlockManagerMaster + +private[spark] +case class RegisterBlockManager( + blockManagerId: BlockManagerId, + maxMemSize: Long, + sender: ActorRef) + extends ToBlockManagerMaster + +private[spark] +case class HeartBeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster + +private[spark] +class UpdateBlockInfo( + var blockManagerId: BlockManagerId, + var blockId: String, + var storageLevel: StorageLevel, + var memSize: Long, + var diskSize: Long) + extends ToBlockManagerMaster + with Externalizable { + + def this() = this(null, null, null, 0, 0) // For deserialization only + + override def writeExternal(out: ObjectOutput) { + blockManagerId.writeExternal(out) + out.writeUTF(blockId) + storageLevel.writeExternal(out) + out.writeInt(memSize.toInt) + out.writeInt(diskSize.toInt) + } + + override def readExternal(in: ObjectInput) { + blockManagerId = new BlockManagerId() + blockManagerId.readExternal(in) + blockId = in.readUTF() + storageLevel = new StorageLevel() + storageLevel.readExternal(in) + memSize = in.readInt() + diskSize = in.readInt() + } +} + +private[spark] +object UpdateBlockInfo { + def apply(blockManagerId: BlockManagerId, + blockId: String, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long): UpdateBlockInfo = { + new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) + } + + // For pattern-matching + def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = { + Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) + } +} + +private[spark] +case class GetLocations(blockId: String) extends ToBlockManagerMaster + +private[spark] +case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster + +private[spark] +case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster + +private[spark] +case class RemoveHost(host: String) extends ToBlockManagerMaster + +private[spark] +case object StopBlockManagerMaster extends ToBlockManagerMaster + +private[spark] +case object GetMemoryStatus extends ToBlockManagerMaster + +private[spark] +case object ExpireDeadHosts extends ToBlockManagerMaster diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala new file mode 100644 index 0000000000000000000000000000000000000000..f570cdc52dd1b2347b8cca8eab62af80b318815e --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala @@ -0,0 +1,16 @@ +package spark.storage + +import akka.actor.Actor + +import spark.{Logging, SparkException, Utils} + + +/** + * An actor to take commands from the master to execute options. For example, + * this is used to remove blocks from the slave's BlockManager. + */ +class BlockManagerSlaveActor(blockManager: BlockManager) extends Actor { + override def receive = { + case RemoveBlock(blockId) => blockManager.removeBlock(blockId) + } +} diff --git a/core/src/main/scala/spark/storage/BlockStore.scala b/core/src/main/scala/spark/storage/BlockStore.scala index 096bf8bdd967c05f5a9e55de844a2084b8a681be..8188d3595eda6762b965beb0711713093038fb5c 100644 --- a/core/src/main/scala/spark/storage/BlockStore.scala +++ b/core/src/main/scala/spark/storage/BlockStore.scala @@ -31,7 +31,12 @@ abstract class BlockStore(val blockManager: BlockManager) extends Logging { def getValues(blockId: String): Option[Iterator[Any]] - def remove(blockId: String) + /** + * Remove a block, if it exists. + * @param blockId the block to remove. + * @return True if the block was found and removed, False otherwise. + */ + def remove(blockId: String): Boolean def contains(blockId: String): Boolean diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 8ba64e4b76a0e142307932da74b0f058833d0ac1..7e5b820cbbdc6ca145c2eb7c6787bd2c137c80d0 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -10,6 +10,8 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import scala.collection.mutable.ArrayBuffer +import spark.executor.ExecutorExitCode + import spark.Utils /** @@ -90,10 +92,13 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes)) } - override def remove(blockId: String) { + override def remove(blockId: String): Boolean = { val file = getFile(blockId) if (file.exists()) { file.delete() + true + } else { + false } } @@ -162,7 +167,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) if (!foundLocalDir) { logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + " attempts to create local dir in " + rootDir) - System.exit(1) + System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } logInfo("Created local directory at " + localDir) localDir diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala index a222b2f3df829f8ad2db0e36e28c133345c7863d..ae88ff0bb1694c89a580bdd89bf127991ecb7f1e 100644 --- a/core/src/main/scala/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/spark/storage/MemoryStore.scala @@ -89,7 +89,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def remove(blockId: String) { + override def remove(blockId: String): Boolean = { entries.synchronized { val entry = entries.get(blockId) if (entry != null) { @@ -97,8 +97,9 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) currentMemory -= entry.size logInfo("Block %s of size %d dropped from memory (free %d)".format( blockId, entry.size, freeMemory)) + true } else { - logWarning("Block " + blockId + " could not be removed as it does not exist") + false } } } diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index eb88eb27597dffab6ffc8dde97629c4c3c8739d7..e3544e5aae28ca5d41ba524ed980e5af98c4f3d7 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -1,9 +1,6 @@ package spark.storage -import java.io.{IOException, Externalizable, ObjectInput, ObjectOutput} -import collection.mutable -import util.Random -import collection.mutable.ArrayBuffer +import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, @@ -13,13 +10,14 @@ import collection.mutable.ArrayBuffer * commonly useful storage levels. */ class StorageLevel( - var useDisk: Boolean, + var useDisk: Boolean, var useMemory: Boolean, var deserialized: Boolean, var replication: Int = 1) extends Externalizable { // TODO: Also add fields for caching priority, dataset ID, and flushing. + assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") def this(flags: Int, replication: Int) { @@ -31,20 +29,16 @@ class StorageLevel( override def clone(): StorageLevel = new StorageLevel( this.useDisk, this.useMemory, this.deserialized, this.replication) - override def hashCode(): Int = { - toInt * 41 + replication - } - override def equals(other: Any): Boolean = other match { case s: StorageLevel => - s.useDisk == useDisk && + s.useDisk == useDisk && s.useMemory == useMemory && s.deserialized == deserialized && - s.replication == replication + s.replication == replication case _ => false } - + def isValid = ((useMemory || useDisk) && (replication > 0)) def toInt: Int = { @@ -75,14 +69,15 @@ class StorageLevel( } @throws(classOf[IOException]) - private def readResolve(): Object = { - StorageLevel.getCachedStorageLevel(this) - } + private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) override def toString: String = "StorageLevel(%b, %b, %b, %d)".format(useDisk, useMemory, deserialized, replication) + + override def hashCode(): Int = toInt * 41 + replication } + object StorageLevel { val NONE = new StorageLevel(false, false, false) val DISK_ONLY = new StorageLevel(true, false, false) @@ -96,9 +91,10 @@ object StorageLevel { val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false) val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2) + private[spark] val storageLevelCache = new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() - def getCachedStorageLevel(level: StorageLevel): StorageLevel = { + private[spark] def getCachedStorageLevel(level: StorageLevel): StorageLevel = { if (storageLevelCache.containsKey(level)) { storageLevelCache.get(level) } else { diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index e4a5b8ffdf6953f37d1ef4cb0b49aa09edab2684..689f07b9692fbf5c68b78080f85222d86fbe8582 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -58,8 +58,10 @@ private[spark] object ThreadingTest { val startTime = System.currentTimeMillis() manager.get(blockId) match { case Some(retrievedBlock) => - assert(retrievedBlock.toList.asInstanceOf[List[Int]] == block.toList, "Block " + blockId + " did not match") - println("Got block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") + assert(retrievedBlock.toList.asInstanceOf[List[Int]] == block.toList, + "Block " + blockId + " did not match") + println("Got block " + blockId + " in " + + (System.currentTimeMillis - startTime) + " ms") case None => assert(false, "Block " + blockId + " could not be retrieved") } @@ -73,8 +75,10 @@ private[spark] object ThreadingTest { System.setProperty("spark.kryoserializer.buffer.mb", "1") val actorSystem = ActorSystem("test") val serializer = new KryoSerializer - val blockManagerMaster = new BlockManagerMaster(actorSystem, true, true) - val blockManager = new BlockManager(blockManagerMaster, serializer, 1024 * 1024) + val masterIp: String = System.getProperty("spark.master.host", "localhost") + val masterPort: Int = System.getProperty("spark.master.port", "7077").toInt + val blockManagerMaster = new BlockManagerMaster(actorSystem, true, true, masterIp, masterPort) + val blockManager = new BlockManager(actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) @@ -86,6 +90,7 @@ private[spark] object ThreadingTest { actorSystem.shutdown() actorSystem.awaitTermination() println("Everything stopped.") - println("It will take sometime for the JVM to clean all temporary files and shutdown. Sit tight.") + println( + "It will take sometime for the JVM to clean all temporary files and shutdown. Sit tight.") } } diff --git a/core/src/main/scala/spark/util/IdGenerator.scala b/core/src/main/scala/spark/util/IdGenerator.scala new file mode 100644 index 0000000000000000000000000000000000000000..b6e309fe1ae68f7c06361dc12f0edbce5f5c3f9a --- /dev/null +++ b/core/src/main/scala/spark/util/IdGenerator.scala @@ -0,0 +1,14 @@ +package spark.util + +import java.util.concurrent.atomic.AtomicInteger + +/** + * A util used to get a unique generation ID. This is a wrapper around Java's + * AtomicInteger. An example usage is in BlockManager, where each BlockManager + * instance would start an Akka actor and we use this utility to assign the Akka + * actors unique names. + */ +private[spark] class IdGenerator { + private var id = new AtomicInteger + def next: Int = id.incrementAndGet +} diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index 2541b2625502c78c77b1dd082d52c24dfa39c3c7..139e21d09e00c6494c7b728967c50266b82d2e30 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -4,24 +4,29 @@ 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 = MetadataCleaner.getDelaySeconds val periodSeconds = math.max(10, delaySeconds / 10) val timer = new Timer(name + " cleanup timer", true) + val task = new TimerTask { def run() { try { if (delaySeconds > 0) { cleanupFunc(System.currentTimeMillis() - (delaySeconds * 1000)) logInfo("Ran metadata cleaner for " + name) - } + } } catch { case e: Exception => logError("Error running cleanup task for " + name, e) } } } + if (periodSeconds > 0) { - logInfo("Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds and " + logInfo( + "Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds and " + "period of " + periodSeconds + " secs") timer.schedule(task, periodSeconds * 1000, periodSeconds * 1000) } @@ -31,7 +36,9 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging } } + object MetadataCleaner { def getDelaySeconds = (System.getProperty("spark.cleaner.delay", "-100").toDouble * 60).toInt def setDelaySeconds(delay: Long) { System.setProperty("spark.cleaner.delay", delay.toString) } } + diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index 52f03784db5c2440c7d29517634ff0b54a7350a5..7e785182eaa86ad3d993b81de23555588f6f71a5 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -1,8 +1,8 @@ package spark.util +import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Map -import java.util.concurrent.ConcurrentHashMap /** * This is a custom implementation of scala.collection.mutable.Map which stores the insertion diff --git a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html index 3dcba3a5457f1d53d10a80ca6b80f7d8ffdea59f..be69e9bf028b0678d7f6acdaa8cb3b131cee4575 100644 --- a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html +++ b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html @@ -4,9 +4,10 @@ <tr> <td> - <a href="http://@worker.host:@worker.webUiPort">@worker.id</href> + <a href="@worker.webUiAddress">@worker.id</href> </td> <td>@{worker.host}:@{worker.port}</td> + <td>@worker.state</td> <td>@worker.cores (@worker.coresUsed Used)</td> <td>@{Utils.memoryMegabytesToString(worker.memory)} (@{Utils.memoryMegabytesToString(worker.memoryUsed)} Used)</td> diff --git a/core/src/main/twirl/spark/deploy/master/worker_table.scala.html b/core/src/main/twirl/spark/deploy/master/worker_table.scala.html index fad1af41dc0619cb9baeed01d9d5a8e7f3f204ac..b249411a62551a76ae5b159c88081e963cca66b6 100644 --- a/core/src/main/twirl/spark/deploy/master/worker_table.scala.html +++ b/core/src/main/twirl/spark/deploy/master/worker_table.scala.html @@ -5,6 +5,7 @@ <tr> <th>ID</th> <th>Address</th> + <th>State</th> <th>Cores</th> <th>Memory</th> </tr> diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 41d84cb01c9dcf5fb21d00bcb73bc6ddd2bcdd2b..302a95db664fbbe8aa7ead5fa91f6ebf68e6a5f5 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -67,7 +67,7 @@ class CheckpointSuite extends FunSuite with BeforeAndAfter with Logging { testCheckpointing(_.glom()) testCheckpointing(_.mapPartitions(_.map(_.toString))) testCheckpointing(r => new MapPartitionsWithSplitRDD(r, - (i: Int, iter: Iterator[Int]) => iter.map(_.toString) )) + (i: Int, iter: Iterator[Int]) => iter.map(_.toString), false)) testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString), 1000) testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x), 1000) testCheckpointing(_.pipe(Seq("cat"))) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 6bd9836a936510cd7a0756d07dde6aee0fffab79..46a0b68f890de7bcccef9c110494244100c2ba1a 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -1,5 +1,12 @@ package spark; +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.util.*; + +import scala.Tuple2; + import com.google.common.base.Charsets; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; @@ -12,8 +19,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import scala.Tuple2; - import spark.api.java.JavaDoubleRDD; import spark.api.java.JavaPairRDD; import spark.api.java.JavaRDD; @@ -24,10 +29,6 @@ import spark.partial.PartialResult; import spark.storage.StorageLevel; import spark.util.StatCounter; -import java.io.File; -import java.io.IOException; -import java.io.Serializable; -import java.util.*; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -383,7 +384,8 @@ public class JavaAPISuite implements Serializable { @Test public void iterator() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0)).next().intValue()); + TaskContext context = new TaskContext(0, 0, 0); + Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue()); } @Test @@ -555,4 +557,17 @@ public class JavaAPISuite implements Serializable { } }).collect().toString()); } + + @Test + public void zip() { + JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + JavaDoubleRDD doubles = rdd.map(new DoubleFunction<Integer>() { + @Override + public Double call(Integer x) { + return 1.0 * x; + } + }); + JavaPairRDD<Integer, Double> zipped = rdd.zip(doubles); + zipped.count(); + } } diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index 4e9717d8718a22594998cde5d038044703777dca..5b4b19896046d204451ad33ad9fc8b0662c6d082 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -2,10 +2,14 @@ package spark import org.scalatest.FunSuite +import akka.actor._ +import spark.scheduler.MapStatus +import spark.storage.BlockManagerId + class MapOutputTrackerSuite extends FunSuite { test("compressSize") { assert(MapOutputTracker.compressSize(0L) === 0) - assert(MapOutputTracker.compressSize(1L) === 0) + assert(MapOutputTracker.compressSize(1L) === 1) assert(MapOutputTracker.compressSize(2L) === 8) assert(MapOutputTracker.compressSize(10L) === 25) assert((MapOutputTracker.compressSize(1000000L) & 0xFF) === 145) @@ -15,11 +19,58 @@ class MapOutputTrackerSuite extends FunSuite { } test("decompressSize") { - assert(MapOutputTracker.decompressSize(0) === 1) + assert(MapOutputTracker.decompressSize(0) === 0) for (size <- Seq(2L, 10L, 100L, 50000L, 1000000L, 1000000000L)) { val size2 = MapOutputTracker.decompressSize(MapOutputTracker.compressSize(size)) assert(size2 >= 0.99 * size && size2 <= 1.11 * size, "size " + size + " decompressed to " + size2 + ", which is out of range") } } + + test("master start and stop") { + val actorSystem = ActorSystem("test") + val tracker = new MapOutputTracker(actorSystem, true) + tracker.stop() + } + + test("master register and fetch") { + val actorSystem = ActorSystem("test") + val tracker = new MapOutputTracker(actorSystem, true) + tracker.registerShuffle(10, 2) + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val compressedSize10000 = MapOutputTracker.compressSize(10000L) + val size1000 = MapOutputTracker.decompressSize(compressedSize1000) + val size10000 = MapOutputTracker.decompressSize(compressedSize10000) + tracker.registerMapOutput(10, 0, new MapStatus(new BlockManagerId("hostA", 1000), + Array(compressedSize1000, compressedSize10000))) + tracker.registerMapOutput(10, 1, new MapStatus(new BlockManagerId("hostB", 1000), + Array(compressedSize10000, compressedSize1000))) + val statuses = tracker.getServerStatuses(10, 0) + assert(statuses.toSeq === Seq((new BlockManagerId("hostA", 1000), size1000), + (new BlockManagerId("hostB", 1000), size10000))) + tracker.stop() + } + + test("master register and unregister and fetch") { + val actorSystem = ActorSystem("test") + val tracker = new MapOutputTracker(actorSystem, true) + tracker.registerShuffle(10, 2) + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val compressedSize10000 = MapOutputTracker.compressSize(10000L) + val size1000 = MapOutputTracker.decompressSize(compressedSize1000) + val size10000 = MapOutputTracker.decompressSize(compressedSize10000) + tracker.registerMapOutput(10, 0, new MapStatus(new BlockManagerId("hostA", 1000), + Array(compressedSize1000, compressedSize1000, compressedSize1000))) + tracker.registerMapOutput(10, 1, new MapStatus(new BlockManagerId("hostB", 1000), + Array(compressedSize10000, compressedSize1000, compressedSize1000))) + + // As if we had two simulatenous fetch failures + tracker.unregisterMapOutput(10, 0, new BlockManagerId("hostA", 1000)) + tracker.unregisterMapOutput(10, 0, new BlockManagerId("hostA", 1000)) + + // The remaining reduce task might try to grab the output dispite the shuffle failure; + // this should cause it to fail, and the scheduler will ignore the failure due to the + // stage already being aborted. + intercept[Exception] { tracker.getServerStatuses(10, 1) } + } } diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 8ac7c8451ad3788f4e7fc53d931dbb24c8f91590..4614901d78684b725c36f2943e34be9c481be68f 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -127,4 +127,16 @@ class RDDSuite extends FunSuite with BeforeAndAfter { assert(coalesced4.glom().collect().map(_.toList).toList === (1 to 10).map(x => List(x)).toList) } + + test("zipped RDDs") { + sc = new SparkContext("local", "test") + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val zipped = nums.zip(nums.map(_ + 1.0)) + assert(zipped.glom().map(_.toList).collect().toList === + List(List((1, 2.0), (2, 3.0)), List((3, 4.0), (4, 5.0)))) + + intercept[IllegalArgumentException] { + nums.zip(sc.parallelize(1 to 4, 1)).collect() + } + } } diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index a2d5e39859ef775114674d4549f37c3e37cc9cf1..8f86e3170ed0f1b0af2874621ac5221a822dbcea 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -7,6 +7,10 @@ import akka.actor._ import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import org.scalatest.PrivateMethodTester +import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.Timeouts._ +import org.scalatest.matchers.ShouldMatchers._ +import org.scalatest.time.SpanSugar._ import spark.KryoSerializer import spark.SizeEstimator @@ -14,23 +18,25 @@ import spark.util.ByteBufferInputStream class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { var store: BlockManager = null + var store2: BlockManager = null var actorSystem: ActorSystem = null var master: BlockManagerMaster = null var oldArch: String = null var oldOops: String = null - - // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test + var oldHeartBeat: String = null + + // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test System.setProperty("spark.kryoserializer.buffer.mb", "1") val serializer = new KryoSerializer before { - actorSystem = ActorSystem("test") - master = new BlockManagerMaster(actorSystem, true, true) + master = new BlockManagerMaster(actorSystem, true, true, "localhost", 7077) - // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case + // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") oldOops = System.setProperty("spark.test.useCompressedOops", "true") + oldHeartBeat = System.setProperty("spark.storage.disableBlockManagerHeartBeat", "true") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() } @@ -38,6 +44,11 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT after { if (store != null) { store.stop() + store = null + } + if (store2 != null) { + store2.stop() + store2 = null } actorSystem.shutdown() actorSystem.awaitTermination() @@ -84,7 +95,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 1 manager interaction") { - store = new BlockManager(master, serializer, 2000) + store = new BlockManager(actorSystem, master, serializer, 2000) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -94,16 +105,16 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY, false) - // Checking whether blocks are in memory + // Checking whether blocks are in memory assert(store.getSingle("a1") != None, "a1 was not in store") assert(store.getSingle("a2") != None, "a2 was not in store") assert(store.getSingle("a3") != None, "a3 was not in store") // Checking whether master knows about the blocks or not - assert(master.getLocations("a1").size === 1, "master was not told about a1") - assert(master.getLocations("a2").size === 1, "master was not told about a2") + assert(master.getLocations("a1").size > 0, "master was not told about a1") + assert(master.getLocations("a2").size > 0, "master was not told about a2") assert(master.getLocations("a3").size === 0, "master was told about a3") - + // Drop a1 and a2 from memory; this should be reported back to the master store.dropFromMemory("a1", null) store.dropFromMemory("a2", null) @@ -114,23 +125,106 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 2 managers interaction") { - store = new BlockManager(master, serializer, 2000) - val otherStore = new BlockManager(master, new KryoSerializer, 2000) + store = new BlockManager(actorSystem, master, serializer, 2000) + store2 = new BlockManager(actorSystem, master, new KryoSerializer, 2000) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") - assert(peers.head === otherStore.blockManagerId, "peer returned by master is not the other manager") + assert(peers.head === store2.blockManagerId, "peer returned by master is not the other manager") val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_2) - otherStore.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_2) + store2.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_2) assert(master.getLocations("a1").size === 2, "master did not report 2 locations for a1") assert(master.getLocations("a2").size === 2, "master did not report 2 locations for a2") } + test("removing block") { + store = new BlockManager(actorSystem, master, serializer, 2000) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + + // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 + store.putSingle("a1-to-remove", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("a2-to-remove", a2, StorageLevel.MEMORY_ONLY) + store.putSingle("a3-to-remove", a3, StorageLevel.MEMORY_ONLY, false) + + // Checking whether blocks are in memory and memory size + val memStatus = master.getMemoryStatus.head._2 + assert(memStatus._1 == 2000L, "total memory " + memStatus._1 + " should equal 2000") + assert(memStatus._2 <= 1200L, "remaining memory " + memStatus._2 + " should <= 1200") + assert(store.getSingle("a1-to-remove") != None, "a1 was not in store") + assert(store.getSingle("a2-to-remove") != None, "a2 was not in store") + assert(store.getSingle("a3-to-remove") != None, "a3 was not in store") + + // Checking whether master knows about the blocks or not + assert(master.getLocations("a1-to-remove").size > 0, "master was not told about a1") + assert(master.getLocations("a2-to-remove").size > 0, "master was not told about a2") + assert(master.getLocations("a3-to-remove").size === 0, "master was told about a3") + + // Remove a1 and a2 and a3. Should be no-op for a3. + master.removeBlock("a1-to-remove") + master.removeBlock("a2-to-remove") + master.removeBlock("a3-to-remove") + + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + store.getSingle("a1-to-remove") should be (None) + master.getLocations("a1-to-remove") should have size 0 + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + store.getSingle("a2-to-remove") should be (None) + master.getLocations("a2-to-remove") should have size 0 + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + store.getSingle("a3-to-remove") should not be (None) + master.getLocations("a3-to-remove") should have size 0 + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + val memStatus = master.getMemoryStatus.head._2 + memStatus._1 should equal (2000L) + memStatus._2 should equal (2000L) + } + } + + test("reregistration on heart beat") { + val heartBeat = PrivateMethod[Unit]('heartBeat) + store = new BlockManager(actorSystem, master, serializer, 2000) + val a1 = new Array[Byte](400) + + store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) + + assert(store.getSingle("a1") != None, "a1 was not in store") + assert(master.getLocations("a1").size > 0, "master was not told about a1") + + master.notifyADeadHost(store.blockManagerId.ip) + assert(master.getLocations("a1").size == 0, "a1 was not removed from master") + + store invokePrivate heartBeat() + assert(master.getLocations("a1").size > 0, "a1 was not reregistered with master") + } + + test("reregistration on block update") { + store = new BlockManager(actorSystem, master, serializer, 2000) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + + store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) + + assert(master.getLocations("a1").size > 0, "master was not told about a1") + + master.notifyADeadHost(store.blockManagerId.ip) + assert(master.getLocations("a1").size == 0, "a1 was not removed from master") + + store.putSingle("a2", a1, StorageLevel.MEMORY_ONLY) + + assert(master.getLocations("a1").size > 0, "a1 was not reregistered with master") + assert(master.getLocations("a2").size > 0, "master was not told about a2") + } + test("in-memory LRU storage") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -147,9 +241,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.getSingle("a2") != None, "a2 was not in store") assert(store.getSingle("a3") === None, "a3 was in store") } - + test("in-memory LRU storage with serialization") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -168,7 +262,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -187,7 +281,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) store.putSingle("rdd_0_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle("rdd_0_2", new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle("rdd_1_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY) @@ -210,7 +304,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("on-disk storage") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -223,7 +317,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -238,7 +332,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with getLocalBytes") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -253,7 +347,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -268,7 +362,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -283,7 +377,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -308,7 +402,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU with streams") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -332,7 +426,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels and streams") { - store = new BlockManager(master, serializer, 1200) + store = new BlockManager(actorSystem, master, serializer, 1200) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -378,7 +472,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("overly large block") { - store = new BlockManager(master, serializer, 500) + store = new BlockManager(actorSystem, master, serializer, 500) store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) @@ -389,49 +483,49 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block compression") { try { System.setProperty("spark.shuffle.compress", "true") - store = new BlockManager(master, serializer, 2000) + store = new BlockManager(actorSystem, master, serializer, 2000) store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("shuffle_0_0_0") <= 100, "shuffle_0_0_0 was not compressed") store.stop() store = null System.setProperty("spark.shuffle.compress", "false") - store = new BlockManager(master, serializer, 2000) + store = new BlockManager(actorSystem, master, serializer, 2000) store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("shuffle_0_0_0") >= 1000, "shuffle_0_0_0 was compressed") store.stop() store = null System.setProperty("spark.broadcast.compress", "true") - store = new BlockManager(master, serializer, 2000) + store = new BlockManager(actorSystem, master, serializer, 2000) store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("broadcast_0") <= 100, "broadcast_0 was not compressed") store.stop() store = null System.setProperty("spark.broadcast.compress", "false") - store = new BlockManager(master, serializer, 2000) + store = new BlockManager(actorSystem, master, serializer, 2000) store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("broadcast_0") >= 1000, "broadcast_0 was compressed") store.stop() store = null System.setProperty("spark.rdd.compress", "true") - store = new BlockManager(master, serializer, 2000) + store = new BlockManager(actorSystem, master, serializer, 2000) store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("rdd_0_0") <= 100, "rdd_0_0 was not compressed") store.stop() store = null System.setProperty("spark.rdd.compress", "false") - store = new BlockManager(master, serializer, 2000) + store = new BlockManager(actorSystem, master, serializer, 2000) store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize("rdd_0_0") >= 1000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager(master, serializer, 2000) + store = new BlockManager(actorSystem, master, serializer, 2000) store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") store.stop() diff --git a/docs/quick-start.md b/docs/quick-start.md index dbc232b6e0424e38c2d04e3576fdcb3a7844da48..177cb145516ce2282c5b45483407c669ff5bf4c7 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -113,7 +113,7 @@ import SparkContext._ object SimpleJob extends Application { val logFile = "/var/log/syslog" // Should be some file on your system val sc = new SparkContext("local", "Simple Job", "$YOUR_SPARK_HOME", - "target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar") + List("target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar")) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() @@ -172,7 +172,7 @@ public class SimpleJob { public static void main(String[] args) { String logFile = "/var/log/syslog"; // Should be some file on your system JavaSparkContext sc = new JavaSparkContext("local", "Simple Job", - "$YOUR_SPARK_HOME", "target/simple-project-1.0.jar"); + "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"}); JavaRDD<String> logData = sc.textFile(logFile).cache(); long numAs = logData.filter(new Function<String, Boolean>() { diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index ae630a037184017a0b4f98b99a8121747a985174..e0ba7c35cb1fa4f21d77030e5c4fcb0734a13943 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -68,7 +68,7 @@ Finally, the following configuration options can be passed to the master and wor 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*. -In `conf/spark-env.sh`, you can set the following parameters, in addition to the [standard Spark configuration settongs](configuration.html): +In `conf/spark-env.sh`, you can set the following parameters, in addition to the [standard Spark configuration settings](configuration.html): <table class="table"> <tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr> diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 2ab11dbd3436276837f4d8c05595662812021966..a5384d3bda1c61d38cec2f4a6d2e70ff1fd84938 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -30,6 +30,7 @@ import time import urllib2 from optparse import OptionParser from sys import stderr +import boto from boto.ec2.blockdevicemapping import BlockDeviceMapping, EBSBlockDeviceType from boto import ec2 @@ -85,6 +86,8 @@ def parse_args(): help="'mesos' for a mesos cluster, 'standalone' for a standalone spark cluster (default: mesos)") parser.add_option("-u", "--user", default="root", help="The ssh user you want to connect as (default: root)") + parser.add_option("--delete-groups", action="store_true", default=False, + help="When destroying a cluster, also destroy the security groups that were created") (opts, args) = parser.parse_args() if len(args) != 2: @@ -283,16 +286,17 @@ def launch_cluster(conn, opts, cluster_name): slave_nodes = [] for zone in zones: num_slaves_this_zone = get_partition(opts.slaves, num_zones, i) - slave_res = image.run(key_name = opts.key_pair, - security_groups = [slave_group], - instance_type = opts.instance_type, - placement = zone, - min_count = num_slaves_this_zone, - max_count = num_slaves_this_zone, - block_device_map = block_map) - slave_nodes += slave_res.instances - print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone, - zone, slave_res.id) + if num_slaves_this_zone > 0: + slave_res = image.run(key_name = opts.key_pair, + security_groups = [slave_group], + instance_type = opts.instance_type, + placement = zone, + min_count = num_slaves_this_zone, + max_count = num_slaves_this_zone, + block_device_map = block_map) + slave_nodes += slave_res.instances + print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone, + zone, slave_res.id) i += 1 # Launch masters @@ -361,6 +365,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k print "Copying SSH key %s to master..." % opts.identity_file ssh(master, opts, 'mkdir -p ~/.ssh') scp(master, opts, opts.identity_file, '~/.ssh/id_rsa') + ssh(master, opts, 'chmod 600 ~/.ssh/id_rsa') print "Running setup on master..." if opts.cluster_type == "mesos": setup_mesos_cluster(master, opts) @@ -555,20 +560,48 @@ def main(): print "Terminating zoo..." for inst in zoo_nodes: inst.terminate() + # Delete security groups as well - group_names = [cluster_name + "-master", cluster_name + "-slaves", cluster_name + "-zoo"] - groups = conn.get_all_security_groups() - for group in groups: - if group.name in group_names: - print "Deleting security group " + group.name - # Delete individual rules before deleting group to remove dependencies - for rule in group.rules: - for grant in rule.grants: - group.revoke(ip_protocol=rule.ip_protocol, - from_port=rule.from_port, - to_port=rule.to_port, - src_group=grant) - conn.delete_security_group(group.name) + if opts.delete_groups: + print "Deleting security groups (this will take some time)..." + group_names = [cluster_name + "-master", cluster_name + "-slaves", cluster_name + "-zoo"] + + attempt = 1; + while attempt <= 3: + print "Attempt %d" % attempt + groups = [g for g in conn.get_all_security_groups() if g.name in group_names] + success = True + # Delete individual rules in all groups before deleting groups to + # remove dependencies between them + for group in groups: + print "Deleting rules in security group " + group.name + for rule in group.rules: + for grant in rule.grants: + success &= group.revoke(ip_protocol=rule.ip_protocol, + from_port=rule.from_port, + to_port=rule.to_port, + src_group=grant) + + # Sleep for AWS eventual-consistency to catch up, and for instances + # to terminate + time.sleep(30) # Yes, it does have to be this long :-( + for group in groups: + try: + conn.delete_security_group(group.name) + print "Deleted security group " + group.name + except boto.exception.EC2ResponseError: + success = False; + print "Failed to delete security group " + group.name + + # Unfortunately, group.revoke() returns True even if a rule was not + # deleted, so this needs to be rerun if something fails + if success: break; + + attempt += 1 + + if not success: + print "Failed to delete all security groups after 3 tries." + print "Try re-running in a few minutes." elif action == "login": (master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster( diff --git a/examples/pom.xml b/examples/pom.xml new file mode 100644 index 0000000000000000000000000000000000000000..782c026d736b968371c1200e4d0f4d69ae1fabcb --- /dev/null +++ b/examples/pom.xml @@ -0,0 +1,106 @@ +<?xml version="1.0" encoding="UTF-8"?> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <parent> + <groupId>org.spark-project</groupId> + <artifactId>parent</artifactId> + <version>0.7.0-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <groupId>org.spark-project</groupId> + <artifactId>spark-examples</artifactId> + <packaging>jar</packaging> + <name>Spark Project Examples</name> + <url>http://spark-project.org/</url> + + <dependencies> + <dependency> + <groupId>org.eclipse.jetty</groupId> + <artifactId>jetty-server</artifactId> + </dependency> + + <dependency> + <groupId>org.scalatest</groupId> + <artifactId>scalatest_${scala.version}</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.scalacheck</groupId> + <artifactId>scalacheck_${scala.version}</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + <build> + <outputDirectory>target/scala-${scala.version}/classes</outputDirectory> + <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory> + <plugins> + <plugin> + <groupId>org.scalatest</groupId> + <artifactId>scalatest-maven-plugin</artifactId> + </plugin> + </plugins> + </build> + + <profiles> + <profile> + <id>hadoop1</id> + <dependencies> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-core</artifactId> + <version>${project.version}</version> + <classifier>hadoop1</classifier> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>provided</scope> + </dependency> + </dependencies> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-jar-plugin</artifactId> + <configuration> + <classifier>hadoop1</classifier> + </configuration> + </plugin> + </plugins> + </build> + </profile> + <profile> + <id>hadoop2</id> + <dependencies> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-core</artifactId> + <version>${project.version}</version> + <classifier>hadoop2</classifier> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-client</artifactId> + <scope>provided</scope> + </dependency> + </dependencies> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-jar-plugin</artifactId> + <configuration> + <classifier>hadoop2</classifier> + </configuration> + </plugin> + </plugins> + </build> + </profile> + </profiles> +</project> diff --git a/pom.xml b/pom.xml new file mode 100644 index 0000000000000000000000000000000000000000..52a4e9d9328bf0f67615e399d231765f81dadd52 --- /dev/null +++ b/pom.xml @@ -0,0 +1,519 @@ +<?xml version="1.0" encoding="UTF-8"?> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <groupId>org.spark-project</groupId> + <artifactId>parent</artifactId> + <version>0.7.0-SNAPSHOT</version> + <packaging>pom</packaging> + <name>Spark Project Parent POM</name> + <url>http://spark-project.org/</url> + <licenses> + <license> + <name>BSD License</name> + <url>https://github.com/mesos/spark/blob/master/LICENSE</url> + <distribution>repo</distribution> + </license> + </licenses> + <scm> + <connection>scm:git:git@github.com:mesos/spark.git</connection> + <url>scm:git:git@github.com:mesos/spark.git</url> + </scm> + <developers> + <developer> + <id>matei</id> + <name>Matei Zaharia</name> + <email>matei.zaharia@gmail.com</email> + <url>http://www.cs.berkeley.edu/~matei</url> + <organization>U.C. Berkeley Computer Science</organization> + <organizationUrl>http://www.cs.berkeley.edu/</organizationUrl> + </developer> + </developers> + <issueManagement> + <system>github</system> + <url>https://spark-project.atlassian.net/browse/SPARK</url> + </issueManagement> + + <prerequisites> + <maven>3.0.0</maven> + </prerequisites> + + <modules> + <module>core</module> + <module>bagel</module> + <module>examples</module> + <module>repl</module> + <module>repl-bin</module> + </modules> + + <properties> + <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding> + <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding> + + <java.version>1.5</java.version> + <scala.version>2.9.2</scala.version> + <mesos.version>0.9.0-incubating</mesos.version> + <akka.version>2.0.3</akka.version> + <spray.version>1.0-M2.1</spray.version> + <slf4j.version>1.6.1</slf4j.version> + <cdh.version>4.1.2</cdh.version> + </properties> + + <repositories> + <repository> + <id>jboss-repo</id> + <name>JBoss Repository</name> + <url>http://repository.jboss.org/nexus/content/repositories/releases/</url> + <releases> + <enabled>true</enabled> + </releases> + <snapshots> + <enabled>false</enabled> + </snapshots> + </repository> + <repository> + <id>cloudera-repo</id> + <name>Cloudera Repository</name> + <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url> + <releases> + <enabled>true</enabled> + </releases> + <snapshots> + <enabled>false</enabled> + </snapshots> + </repository> + <repository> + <id>typesafe-repo</id> + <name>Typesafe Repository</name> + <url>http://repo.typesafe.com/typesafe/releases/</url> + <releases> + <enabled>true</enabled> + </releases> + <snapshots> + <enabled>false</enabled> + </snapshots> + </repository> + <repository> + <id>spray-repo</id> + <name>Spray Repository</name> + <url>http://repo.spray.cc/</url> + <releases> + <enabled>true</enabled> + </releases> + <snapshots> + <enabled>false</enabled> + </snapshots> + </repository> + </repositories> + <pluginRepositories> + <pluginRepository> + <id>oss-sonatype-releases</id> + <name>OSS Sonatype</name> + <url>https://oss.sonatype.org/content/repositories/releases</url> + <releases> + <enabled>true</enabled> + </releases> + <snapshots> + <enabled>false</enabled> + </snapshots> + </pluginRepository> + <pluginRepository> + <id>oss-sonatype-snapshots</id> + <name>OSS Sonatype</name> + <url>https://oss.sonatype.org/content/repositories/snapshots</url> + <releases> + <enabled>false</enabled> + </releases> + <snapshots> + <enabled>true</enabled> + </snapshots> + </pluginRepository> + <pluginRepository> + <id>oss-sonatype</id> + <name>OSS Sonatype</name> + <url>https://oss.sonatype.org/content/groups/public</url> + <releases> + <enabled>true</enabled> + </releases> + <snapshots> + <enabled>true</enabled> + </snapshots> + </pluginRepository> + </pluginRepositories> + + <dependencyManagement> + <dependencies> + <dependency> + <groupId>org.eclipse.jetty</groupId> + <artifactId>jetty-server</artifactId> + <version>7.5.3.v20111011</version> + </dependency> + <dependency> + <groupId>com.google.guava</groupId> + <artifactId>guava</artifactId> + <version>11.0.1</version> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-api</artifactId> + <version>${slf4j.version}</version> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-log4j12</artifactId> + <version>${slf4j.version}</version> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>jul-to-slf4j</artifactId> + <version>${slf4j.version}</version> + </dependency> + <dependency> + <groupId>com.ning</groupId> + <artifactId>compress-lzf</artifactId> + <version>0.8.4</version> + </dependency> + <dependency> + <groupId>asm</groupId> + <artifactId>asm-all</artifactId> + <version>3.3.1</version> + </dependency> + <dependency> + <groupId>com.google.protobuf</groupId> + <artifactId>protobuf-java</artifactId> + <version>2.4.1</version> + </dependency> + <dependency> + <groupId>de.javakaffee</groupId> + <artifactId>kryo-serializers</artifactId> + <version>0.9</version> + </dependency> + <dependency> + <groupId>com.typesafe.akka</groupId> + <artifactId>akka-actor</artifactId> + <version>${akka.version}</version> + </dependency> + <dependency> + <groupId>com.typesafe.akka</groupId> + <artifactId>akka-remote</artifactId> + <version>${akka.version}</version> + </dependency> + <dependency> + <groupId>com.typesafe.akka</groupId> + <artifactId>akka-slf4j</artifactId> + <version>${akka.version}</version> + </dependency> + <dependency> + <groupId>it.unimi.dsi</groupId> + <artifactId>fastutil</artifactId> + <version>6.4.4</version> + </dependency> + <dependency> + <groupId>colt</groupId> + <artifactId>colt</artifactId> + <version>1.2.0</version> + </dependency> + <dependency> + <groupId>cc.spray</groupId> + <artifactId>spray-can</artifactId> + <version>${spray.version}</version> + </dependency> + <dependency> + <groupId>cc.spray</groupId> + <artifactId>spray-server</artifactId> + <version>${spray.version}</version> + </dependency> + <dependency> + <groupId>org.tomdz.twirl</groupId> + <artifactId>twirl-api</artifactId> + <version>1.0.2</version> + </dependency> + <dependency> + <groupId>com.github.scala-incubator.io</groupId> + <artifactId>scala-io-file_${scala.version}</artifactId> + <version>0.4.1</version> + </dependency> + <dependency> + <groupId>org.apache.mesos</groupId> + <artifactId>mesos</artifactId> + <version>${mesos.version}</version> + </dependency> + + <dependency> + <groupId>org.scala-lang</groupId> + <artifactId>scala-compiler</artifactId> + <version>${scala.version}</version> + </dependency> + <dependency> + <groupId>org.scala-lang</groupId> + <artifactId>jline</artifactId> + <version>${scala.version}</version> + </dependency> + + <dependency> + <groupId>org.scalatest</groupId> + <artifactId>scalatest_${scala.version}</artifactId> + <version>1.8</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.scalacheck</groupId> + <artifactId>scalacheck_${scala.version}</artifactId> + <version>1.9</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>com.novocode</groupId> + <artifactId>junit-interface</artifactId> + <version>0.8</version> + <scope>test</scope> + </dependency> + </dependencies> + </dependencyManagement> + + <build> + <pluginManagement> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-enforcer-plugin</artifactId> + <version>1.1.1</version> + <executions> + <execution> + <id>enforce-versions</id> + <goals> + <goal>enforce</goal> + </goals> + <configuration> + <rules> + <requireMavenVersion> + <version>3.0.0</version> + </requireMavenVersion> + <requireJavaVersion> + <version>${java.version}</version> + </requireJavaVersion> + </rules> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.codehaus.mojo</groupId> + <artifactId>build-helper-maven-plugin</artifactId> + <version>1.7</version> + </plugin> + <plugin> + <groupId>org.tomdz.twirl</groupId> + <artifactId>twirl-maven-plugin</artifactId> + <version>1.0.1</version> + <executions> + <execution> + <goals> + <goal>generate</goal> + </goals> + </execution> + </executions> + </plugin> + <plugin> + <groupId>net.alchim31.maven</groupId> + <artifactId>scala-maven-plugin</artifactId> + <version>3.1.0</version> + <executions> + <execution> + <id>scala-compile-first</id> + <phase>process-resources</phase> + <goals> + <goal>compile</goal> + </goals> + </execution> + <execution> + <id>scala-test-compile-first</id> + <phase>process-test-resources</phase> + <goals> + <goal>testCompile</goal> + </goals> + </execution> + <execution> + <id>attach-scaladocs</id> + <phase>verify</phase> + <goals> + <goal>doc-jar</goal> + </goals> + </execution> + </executions> + <configuration> + <scalaVersion>${scala.version}</scalaVersion> + <recompileMode>incremental</recompileMode> + <args> + <arg>-unchecked</arg> + <arg>-optimise</arg> + </args> + <jvmArgs> + <jvmArg>-Xms64m</jvmArg> + <jvmArg>-Xmx1024m</jvmArg> + </jvmArgs> + <javacArgs> + <javacArg>-source</javacArg> + <javacArg>${java.version}</javacArg> + <javacArg>-target</javacArg> + <javacArg>${java.version}</javacArg> + </javacArgs> + </configuration> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-compiler-plugin</artifactId> + <version>2.5.1</version> + <configuration> + <source>${java.version}</source> + <target>${java.version}</target> + <encoding>UTF-8</encoding> + <maxmem>1024m</maxmem> + </configuration> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-surefire-plugin</artifactId> + <version>2.12.4</version> + <configuration> + <!-- Uses scalatest instead --> + <skipTests>true</skipTests> + </configuration> + </plugin> + <plugin> + <groupId>org.scalatest</groupId> + <artifactId>scalatest-maven-plugin</artifactId> + <version>1.0-M2</version> + <configuration> + <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory> + <junitxml>.</junitxml> + <filereports>WDF TestSuite.txt</filereports> + <argLine>-Xms64m -Xmx1024m</argLine> + </configuration> + <executions> + <execution> + <id>test</id> + <goals> + <goal>test</goal> + </goals> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-jar-plugin</artifactId> + <version>2.4</version> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-antrun-plugin</artifactId> + <version>1.7</version> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-shade-plugin</artifactId> + <version>2.0</version> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-source-plugin</artifactId> + <version>2.2.1</version> + <configuration> + <attach>true</attach> + </configuration> + <executions> + <execution> + <id>create-source-jar</id> + <goals> + <goal>jar-no-fork</goal> + </goals> + </execution> + </executions> + </plugin> + </plugins> + </pluginManagement> + + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-enforcer-plugin</artifactId> + </plugin> + <plugin> + <groupId>org.codehaus.mojo</groupId> + <artifactId>build-helper-maven-plugin</artifactId> + <executions> + <execution> + <id>add-scala-sources</id> + <phase>generate-sources</phase> + <goals> + <goal>add-source</goal> + </goals> + <configuration> + <sources> + <source>src/main/scala</source> + </sources> + </configuration> + </execution> + <execution> + <id>add-scala-test-sources</id> + <phase>generate-test-sources</phase> + <goals> + <goal>add-test-source</goal> + </goals> + <configuration> + <sources> + <source>src/test/scala</source> + </sources> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>net.alchim31.maven</groupId> + <artifactId>scala-maven-plugin</artifactId> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-source-plugin</artifactId> + </plugin> + </plugins> + </build> + + <profiles> + <profile> + <id>hadoop1</id> + <properties> + <hadoop.major.version>1</hadoop.major.version> + </properties> + <dependencyManagement> + <dependencies> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <version>0.20.205.0</version> + </dependency> + </dependencies> + </dependencyManagement> + </profile> + + <profile> + <id>hadoop2</id> + <properties> + <hadoop.major.version>2</hadoop.major.version> + </properties> + <dependencyManagement> + <dependencies> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <version>2.0.0-mr1-cdh${cdh.version}</version> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-client</artifactId> + <version>2.0.0-mr1-cdh${cdh.version}</version> + </dependency> + </dependencies> + </dependencyManagement> + </profile> + </profiles> +</project> diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 05f3c59681ecb5a0ab1d3805c6611b0330bd23a8..b2e0aff4077b673c1935fc8baa43e9bfd1b5fd0a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -89,7 +89,7 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", - "org.scalatest" %% "scalatest" % "1.6.1" % "test", + "org.scalatest" %% "scalatest" % "1.8" % "test", "org.scalacheck" %% "scalacheck" % "1.9" % "test", "com.novocode" % "junit-interface" % "0.8" % "test", "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml new file mode 100644 index 0000000000000000000000000000000000000000..0667b71cc7bba49a22a2c12d7f9c590dc30ea78c --- /dev/null +++ b/repl-bin/pom.xml @@ -0,0 +1,232 @@ +<?xml version="1.0" encoding="UTF-8"?> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <parent> + <groupId>org.spark-project</groupId> + <artifactId>parent</artifactId> + <version>0.7.0-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <groupId>org.spark-project</groupId> + <artifactId>spark-repl-bin</artifactId> + <packaging>pom</packaging> + <name>Spark Project REPL binary packaging</name> + <url>http://spark-project.org/</url> + + <properties> + <deb.pkg.name>spark-${classifier}</deb.pkg.name> + <deb.install.path>/usr/share/spark-${classifier}</deb.install.path> + <deb.user>root</deb.user> + </properties> + + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-shade-plugin</artifactId> + <configuration> + <shadedArtifactAttached>false</shadedArtifactAttached> + <outputFile>${project.build.directory}/${project.artifactId}-${project.version}-shaded-${classifier}.jar</outputFile> + <artifactSet> + <includes> + <include>*:*</include> + </includes> + </artifactSet> + <filters> + <filter> + <artifact>*:*</artifact> + <excludes> + <exclude>META-INF/*.SF</exclude> + <exclude>META-INF/*.DSA</exclude> + <exclude>META-INF/*.RSA</exclude> + </excludes> + </filter> + </filters> + </configuration> + <executions> + <execution> + <phase>package</phase> + <goals> + <goal>shade</goal> + </goals> + <configuration> + <transformers> + <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/> + <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer"> + <resource>reference.conf</resource> + </transformer> + <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer"> + <mainClass>spark.repl.Main</mainClass> + </transformer> + </transformers> + </configuration> + </execution> + </executions> + </plugin> + </plugins> + </build> + + <profiles> + <profile> + <id>hadoop1</id> + <properties> + <classifier>hadoop1</classifier> + </properties> + <dependencies> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-core</artifactId> + <version>${project.version}</version> + <classifier>hadoop1</classifier> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-bagel</artifactId> + <version>${project.version}</version> + <classifier>hadoop1</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-examples</artifactId> + <version>${project.version}</version> + <classifier>hadoop1</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-repl</artifactId> + <version>${project.version}</version> + <classifier>hadoop1</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>runtime</scope> + </dependency> + </dependencies> + </profile> + <profile> + <id>hadoop2</id> + <properties> + <classifier>hadoop2</classifier> + </properties> + <dependencies> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-core</artifactId> + <version>${project.version}</version> + <classifier>hadoop2</classifier> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-bagel</artifactId> + <version>${project.version}</version> + <classifier>hadoop2</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-examples</artifactId> + <version>${project.version}</version> + <classifier>hadoop2</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-repl</artifactId> + <version>${project.version}</version> + <classifier>hadoop2</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-client</artifactId> + <scope>runtime</scope> + </dependency> + </dependencies> + </profile> + <profile> + <id>deb</id> + <build> + <plugins> + <plugin> + <groupId>org.codehaus.mojo</groupId> + <artifactId>buildnumber-maven-plugin</artifactId> + <version>1.1</version> + <executions> + <execution> + <phase>validate</phase> + <goals> + <goal>create</goal> + </goals> + <configuration> + <shortRevisionLength>8</shortRevisionLength> + </configuration> + </execution> + </executions> + </plugin> + <plugin> + <groupId>org.vafer</groupId> + <artifactId>jdeb</artifactId> + <version>0.11</version> + <executions> + <execution> + <phase>package</phase> + <goals> + <goal>jdeb</goal> + </goals> + <configuration> + <deb>${project.build.directory}/${deb.pkg.name}_${project.version}-${buildNumber}_all.deb</deb> + <attach>false</attach> + <compression>gzip</compression> + <dataSet> + <data> + <src>${project.build.directory}/${project.artifactId}-${project.version}-shaded-${classifier}.jar</src> + <type>file</type> + <mapper> + <type>perm</type> + <user>${deb.user}</user> + <group>${deb.user}</group> + <prefix>${deb.install.path}</prefix> + </mapper> + </data> + <data> + <src>${basedir}/src/deb/bin</src> + <type>directory</type> + <mapper> + <type>perm</type> + <user>${deb.user}</user> + <group>${deb.user}</group> + <prefix>${deb.install.path}</prefix> + <filemode>744</filemode> + </mapper> + </data> + <data> + <src>${basedir}/../conf</src> + <type>directory</type> + <mapper> + <type>perm</type> + <user>${deb.user}</user> + <group>${deb.user}</group> + <prefix>${deb.install.path}/conf</prefix> + <filemode>744</filemode> + </mapper> + </data> + </dataSet> + </configuration> + </execution> + </executions> + </plugin> + </plugins> + </build> + </profile> + </profiles> +</project> diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run new file mode 100755 index 0000000000000000000000000000000000000000..c54c9e97a003b71899ea5bd9148908a8f25644f6 --- /dev/null +++ b/repl-bin/src/deb/bin/run @@ -0,0 +1,41 @@ +#!/bin/bash + +SCALA_VERSION=2.9.2 + +# Figure out where the Scala framework is installed +FWDIR="$(cd `dirname $0`; pwd)" + +# Export this as SPARK_HOME +export SPARK_HOME="$FWDIR" + +# Load environment variables from conf/spark-env.sh, if it exists +if [ -e $FWDIR/conf/spark-env.sh ] ; then + . $FWDIR/conf/spark-env.sh +fi + +# Figure out how much memory to use per executor and set it as an environment +# variable so that our process sees it and can report it to Mesos +if [ -z "$SPARK_MEM" ] ; then + SPARK_MEM="512m" +fi +export SPARK_MEM + +# Set JAVA_OPTS to be able to load native libraries and to set heap size +JAVA_OPTS="$SPARK_JAVA_OPTS" +JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH" +JAVA_OPTS+=" -Xms$SPARK_MEM -Xmx$SPARK_MEM" +# Load extra JAVA_OPTS from conf/java-opts, if it exists +if [ -e $FWDIR/conf/java-opts ] ; then + JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" +fi +export JAVA_OPTS + +# Build up classpath +CLASSPATH="$SPARK_CLASSPATH" +CLASSPATH+=":$FWDIR/conf" +for jar in `find $FWDIR -name '*jar'`; do + CLASSPATH+=":$jar" +done +export CLASSPATH + +exec java -Dscala.usejavacp=true -Djline.shutdownhook=true -cp "$CLASSPATH" $JAVA_OPTS $EXTRA_ARGS "$@" diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor new file mode 100755 index 0000000000000000000000000000000000000000..47b9cccdfed8d5243206526f3ce39471020d52db --- /dev/null +++ b/repl-bin/src/deb/bin/spark-executor @@ -0,0 +1,5 @@ +#!/bin/bash + +FWDIR="$(cd `dirname $0`; pwd)" +echo "Running spark-executor with framework dir = $FWDIR" +exec $FWDIR/run spark.executor.MesosExecutorBackend diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell new file mode 100755 index 0000000000000000000000000000000000000000..219c66eb0ba5b6092a07dd16b388a9f72ea4f233 --- /dev/null +++ b/repl-bin/src/deb/bin/spark-shell @@ -0,0 +1,4 @@ +#!/bin/bash + +FWDIR="$(cd `dirname $0`; pwd)" +exec $FWDIR/run spark.repl.Main "$@" diff --git a/repl-bin/src/deb/control/control b/repl-bin/src/deb/control/control new file mode 100644 index 0000000000000000000000000000000000000000..a6b4471d485f475358462b5f9437b104cce46d09 --- /dev/null +++ b/repl-bin/src/deb/control/control @@ -0,0 +1,8 @@ +Package: [[deb.pkg.name]] +Version: [[version]]-[[buildNumber]] +Section: misc +Priority: extra +Architecture: all +Maintainer: Matei Zaharia <matei.zaharia@gmail.com> +Description: [[name]] +Distribution: development diff --git a/repl/pom.xml b/repl/pom.xml new file mode 100644 index 0000000000000000000000000000000000000000..114e3e9932f4eebcf28f012a7d3410560fc2c003 --- /dev/null +++ b/repl/pom.xml @@ -0,0 +1,167 @@ +<?xml version="1.0" encoding="UTF-8"?> +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + <parent> + <groupId>org.spark-project</groupId> + <artifactId>parent</artifactId> + <version>0.7.0-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <groupId>org.spark-project</groupId> + <artifactId>spark-repl</artifactId> + <packaging>jar</packaging> + <name>Spark Project REPL</name> + <url>http://spark-project.org/</url> + + <properties> + <deb.install.path>/usr/share/spark</deb.install.path> + <deb.user>root</deb.user> + </properties> + + <dependencies> + <dependency> + <groupId>org.eclipse.jetty</groupId> + <artifactId>jetty-server</artifactId> + </dependency> + <dependency> + <groupId>org.scala-lang</groupId> + <artifactId>scala-compiler</artifactId> + </dependency> + <dependency> + <groupId>org.scala-lang</groupId> + <artifactId>jline</artifactId> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>jul-to-slf4j</artifactId> + </dependency> + <dependency> + <groupId>org.slf4j</groupId> + <artifactId>slf4j-log4j12</artifactId> + </dependency> + + <dependency> + <groupId>org.scalatest</groupId> + <artifactId>scalatest_${scala.version}</artifactId> + <scope>test</scope> + </dependency> + <dependency> + <groupId>org.scalacheck</groupId> + <artifactId>scalacheck_${scala.version}</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + <build> + <outputDirectory>target/scala-${scala.version}/classes</outputDirectory> + <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory> + <plugins> + <plugin> + <groupId>org.scalatest</groupId> + <artifactId>scalatest-maven-plugin</artifactId> + <configuration> + <environmentVariables> + <SPARK_HOME>${basedir}/..</SPARK_HOME> + <SPARK_TESTING>1</SPARK_TESTING> + </environmentVariables> + </configuration> + </plugin> + </plugins> + </build> + + <profiles> + <profile> + <id>hadoop1</id> + <properties> + <classifier>hadoop1</classifier> + </properties> + <dependencies> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-core</artifactId> + <version>${project.version}</version> + <classifier>hadoop1</classifier> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-bagel</artifactId> + <version>${project.version}</version> + <classifier>hadoop1</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-examples</artifactId> + <version>${project.version}</version> + <classifier>hadoop1</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>provided</scope> + </dependency> + </dependencies> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-jar-plugin</artifactId> + <configuration> + <classifier>hadoop1</classifier> + </configuration> + </plugin> + </plugins> + </build> + </profile> + <profile> + <id>hadoop2</id> + <properties> + <classifier>hadoop2</classifier> + </properties> + <dependencies> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-core</artifactId> + <version>${project.version}</version> + <classifier>hadoop2</classifier> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-bagel</artifactId> + <version>${project.version}</version> + <classifier>hadoop2</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.spark-project</groupId> + <artifactId>spark-examples</artifactId> + <version>${project.version}</version> + <classifier>hadoop2</classifier> + <scope>runtime</scope> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-core</artifactId> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>org.apache.hadoop</groupId> + <artifactId>hadoop-client</artifactId> + <scope>provided</scope> + </dependency> + </dependencies> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-jar-plugin</artifactId> + <configuration> + <classifier>hadoop2</classifier> + </configuration> + </plugin> + </plugins> + </build> + </profile> + </profiles> +</project> diff --git a/run b/run index d91430ad2e0edc983fc0679e130934caabc09001..27506c33e2db431fad02ede8256032925e7b16a1 100755 --- a/run +++ b/run @@ -80,13 +80,18 @@ 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" -for jar in `find $FWDIR/lib_managed/jars -name '*jar'`; do - CLASSPATH+=":$jar" -done -for jar in `find $FWDIR/lib_managed/bundles -name '*jar'`; do +if [ -e "$FWDIR/lib_managed" ]; then + for jar in `find "$FWDIR/lib_managed/jars" -name '*jar'`; do + CLASSPATH+=":$jar" + done + for jar in `find "$FWDIR/lib_managed/bundles" -name '*jar'`; do + CLASSPATH+=":$jar" + done +fi +for jar in `find "$REPL_DIR/lib" -name '*jar'`; do CLASSPATH+=":$jar" done -for jar in `find $REPL_DIR/lib -name '*jar'`; do +for jar in `find "$REPL_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do CLASSPATH+=":$jar" done CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"