From 59577df14c06417676a9ffdd599f5713c448e299 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 9 May 2014 14:51:34 -0700 Subject: [PATCH 01/82] SPARK-1770: Revert accidental(?) fix Looks like this change was accidentally committed here: https://github.com/apache/spark/commit/06b15baab25951d124bbe6b64906f4139e037deb but the change does not show up in the PR itself (#704). Other than not intending to go in with that PR, this also broke the test JavaAPISuite.repartition. Author: Aaron Davidson Closes #716 from aarondav/shufflerand and squashes the following commits: b1cf70b [Aaron Davidson] SPARK-1770: Revert accidental(?) fix --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 9d8d8044f07eb..a1ca612cc9a09 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -330,9 +330,9 @@ abstract class RDD[T: ClassTag]( if (shuffle) { // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[Int, T, (Int, T)](map(x => (Utils.random.nextInt(), x)), + new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), new HashPartitioner(numPartitions)), - numPartitions).values + numPartitions).keys } else { new CoalescedRDD(this, numPartitions) } From 2f452cbaf35dbc609ab48ec0ee5e3dd7b6b9b790 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 9 May 2014 21:50:23 -0700 Subject: [PATCH 02/82] SPARK-1686: keep schedule() calling in the main thread https://issues.apache.org/jira/browse/SPARK-1686 moved from original JIRA (by @markhamstra): In deploy.master.Master, the completeRecovery method is the last thing to be called when a standalone Master is recovering from failure. It is responsible for resetting some state, relaunching drivers, and eventually resuming its scheduling duties. There are currently four places in Master.scala where completeRecovery is called. Three of them are from within the actor's receive method, and aren't problems. The last starts from within receive when the ElectedLeader message is received, but the actual completeRecovery() call is made from the Akka scheduler. That means that it will execute on a different scheduler thread, and Master itself will end up running (i.e., schedule() ) from that Akka scheduler thread. In this PR, I added a new master message TriggerSchedule to trigger the "local" call of schedule() in the scheduler thread Author: CodingCat Closes #639 from CodingCat/SPARK-1686 and squashes the following commits: 81bb4ca [CodingCat] rename variable 69e0a2a [CodingCat] style fix 36a2ac0 [CodingCat] address Aaron's comments ec9b7bb [CodingCat] address the comments 02b37ca [CodingCat] keep schedule() calling in the main thread --- .../org/apache/spark/deploy/master/Master.scala | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index fdb633bd33608..f254f5585ba25 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -104,6 +104,8 @@ private[spark] class Master( var leaderElectionAgent: ActorRef = _ + private var recoveryCompletionTask: Cancellable = _ + // 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 app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. @@ -152,6 +154,10 @@ private[spark] class Master( } override def postStop() { + // prevent the CompleteRecovery message sending to restarted master + if (recoveryCompletionTask != null) { + recoveryCompletionTask.cancel() + } webUi.stop() fileSystemsUsed.foreach(_.close()) masterMetricsSystem.stop() @@ -171,10 +177,13 @@ private[spark] class Master( logInfo("I have been elected leader! New state: " + state) if (state == RecoveryState.RECOVERING) { beginRecovery(storedApps, storedDrivers, storedWorkers) - context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() } + recoveryCompletionTask = context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis, self, + CompleteRecovery) } } + case CompleteRecovery => completeRecovery() + case RevokedLeadership => { logError("Leadership has been revoked -- master shutting down.") System.exit(0) @@ -465,7 +474,7 @@ private[spark] class Master( * Schedule the currently available resources among waiting apps. This method will be called * every time a new app joins or resource availability changes. */ - def schedule() { + private def schedule() { if (state != RecoveryState.ALIVE) { return } // First schedule drivers, they take strict precedence over applications @@ -485,7 +494,7 @@ private[spark] class Master( // Try to spread out each app among all the nodes, until it has all its cores for (app <- waitingApps if app.coresLeft > 0) { val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(canUse(app, _)).sortBy(_.coresFree).reverse + .filter(canUse(app, _)).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(app.coresLeft, usableWorkers.map(_.coresFree).sum) From 561510867a1b79beef57acf9df65c9f88481435d Mon Sep 17 00:00:00 2001 From: witgo Date: Sat, 10 May 2014 10:15:04 -0700 Subject: [PATCH 03/82] [SPARK-1644] The org.datanucleus:* should not be packaged into spark-assembly-*.jar Author: witgo Closes #688 from witgo/SPARK-1644 and squashes the following commits: 56ad6ac [witgo] review commit 87c03e4 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1644 6ffa7e4 [witgo] review commit a597414 [witgo] The org.datanucleus:* should not be packaged into spark-assembly-*.jar --- assembly/pom.xml | 1 + project/SparkBuild.scala | 11 ++++++----- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7d123fb1d7f02..6c4d46aeb67bd 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -96,6 +96,7 @@ *:* + org.datanucleus:* META-INF/*.SF META-INF/*.DSA META-INF/*.RSA diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7f9746ec4acc0..27e9505ec9831 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -579,12 +579,13 @@ object SparkBuild extends Build { def extraAssemblySettings() = Seq( test in assembly := {}, mergeStrategy in assembly := { - case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard - case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard - case "log4j.properties" => MergeStrategy.discard + case PathList("org", "datanucleus", xs @ _*) => MergeStrategy.discard + case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard + case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard + case "log4j.properties" => MergeStrategy.discard case m if m.toLowerCase.startsWith("meta-inf/services/") => MergeStrategy.filterDistinctLines - case "reference.conf" => MergeStrategy.concat - case _ => MergeStrategy.first + case "reference.conf" => MergeStrategy.concat + case _ => MergeStrategy.first } ) From 4d6055329846f5e09472e5f844127a5ab5880e15 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 10 May 2014 11:48:01 -0700 Subject: [PATCH 04/82] [SQL] Upgrade parquet library. I think we are hitting this issue in some perf tests: https://github.com/Parquet/parquet-mr/commit/6aed5288fd4a1398063a5a219b2ae4a9f71b02cf Credit to @aarondav ! Author: Michael Armbrust Closes #684 from marmbrus/upgradeParquet and squashes the following commits: e10a619 [Michael Armbrust] Upgrade parquet library. --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index fb904e868cfaf..e0bff60a54cde 100644 --- a/pom.xml +++ b/pom.xml @@ -123,7 +123,7 @@ ${hadoop.version} 0.94.6 0.12.0 - 1.3.2 + 1.4.3 1.2.3 8.1.14.v20131031 0.3.6 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 27e9505ec9831..af882b3ea7beb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -300,7 +300,7 @@ object SparkBuild extends Build { val jets3tVersion = if ("^2\\.[3-9]+".r.findFirstIn(hadoopVersion).isDefined) "0.9.0" else "0.7.1" val jettyVersion = "8.1.14.v20131031" val hiveVersion = "0.12.0" - val parquetVersion = "1.3.2" + val parquetVersion = "1.4.3" val slf4jVersion = "1.7.5" val excludeNetty = ExclusionRule(organization = "org.jboss.netty") From 8e94d2721a9d3d36697e13f8cc6567ae8aeee78b Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 10 May 2014 12:03:27 -0700 Subject: [PATCH 05/82] [SPARK-1778] [SQL] Add 'limit' transformation to SchemaRDD. Add `limit` transformation to `SchemaRDD`. Author: Takuya UESHIN Closes #711 from ueshin/issues/SPARK-1778 and squashes the following commits: 33169df [Takuya UESHIN] Add 'limit' transformation to SchemaRDD. --- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 9 +++++++++ .../test/scala/org/apache/spark/sql/DslQuerySuite.scala | 6 ++++++ 2 files changed, 15 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 34200be3ac955..2569815ebb209 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -178,6 +178,15 @@ class SchemaRDD( def orderBy(sortExprs: SortOrder*): SchemaRDD = new SchemaRDD(sqlContext, Sort(sortExprs, logicalPlan)) + /** + * Limits the results by the given expressions. + * {{{ + * schemaRDD.limit(10) + * }}} + */ + def limit(limitExpr: Expression): SchemaRDD = + new SchemaRDD(sqlContext, Limit(limitExpr, logicalPlan)) + /** * Performs a grouping followed by an aggregation. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index be0f4a4c73b36..92a707ea57504 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -71,6 +71,12 @@ class DslQuerySuite extends QueryTest { Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) } + test("limit") { + checkAnswer( + testData.limit(10), + testData.take(10).toSeq) + } + test("average") { checkAnswer( testData2.groupBy()(Average('a)), From 7eefc9d2b3f6ebc0ecb5562da7323f1e06afbb35 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 10 May 2014 12:10:24 -0700 Subject: [PATCH 06/82] SPARK-1708. Add a ClassTag on Serializer and things that depend on it This pull request contains a rebased patch from @heathermiller (https://github.com/heathermiller/spark/pull/1) to add ClassTags on Serializer and types that depend on it (Broadcast and AccumulableCollection). Putting these in the public API signatures now will allow us to use Scala Pickling for serialization down the line without breaking binary compatibility. One question remaining is whether we also want them on Accumulator -- Accumulator is passed as part of a bigger Task or TaskResult object via the closure serializer so it doesn't seem super useful to add the ClassTag there. Broadcast and AccumulableCollection in contrast were being serialized directly. CC @rxin, @pwendell, @heathermiller Author: Matei Zaharia Closes #700 from mateiz/spark-1708 and squashes the following commits: 1a3d8b0 [Matei Zaharia] Use fake ClassTag in Java 3b449ed [Matei Zaharia] test fix 2209a27 [Matei Zaharia] Code style fixes 9d48830 [Matei Zaharia] Add a ClassTag on Serializer and things that depend on it --- .../scala/org/apache/spark/Accumulators.scala | 7 +-- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../spark/api/java/JavaSparkContext.scala | 2 +- .../apache/spark/broadcast/Broadcast.scala | 4 +- .../spark/broadcast/BroadcastFactory.scala | 4 +- .../spark/broadcast/BroadcastManager.scala | 4 +- .../spark/broadcast/HttpBroadcast.scala | 7 ++- .../broadcast/HttpBroadcastFactory.scala | 4 +- .../spark/broadcast/TorrentBroadcast.scala | 4 +- .../broadcast/TorrentBroadcastFactory.scala | 4 +- .../org/apache/spark/rdd/CheckpointRDD.scala | 4 +- .../spark/rdd/ParallelCollectionRDD.scala | 2 +- .../apache/spark/rdd/RDDCheckpointData.scala | 2 +- .../spark/serializer/JavaSerializer.scala | 13 +++--- .../spark/serializer/KryoSerializer.scala | 12 ++--- .../apache/spark/serializer/Serializer.scala | 17 +++---- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../serializer/KryoSerializerSuite.scala | 11 ++--- .../bagel/WikipediaPageRankStandalone.scala | 12 ++--- .../spark/graphx/impl/Serializers.scala | 45 ++++++++++--------- .../apache/spark/graphx/SerializerSuite.scala | 5 ++- .../sql/execution/SparkSqlSerializer.scala | 6 ++- 22 files changed, 103 insertions(+), 72 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 6d652faae149a..cdfd338081fa2 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -21,6 +21,7 @@ import java.io.{ObjectInputStream, Serializable} import scala.collection.generic.Growable import scala.collection.mutable.Map +import scala.reflect.ClassTag import org.apache.spark.serializer.JavaSerializer @@ -164,9 +165,9 @@ trait AccumulableParam[R, T] extends Serializable { def zero(initialValue: R): R } -private[spark] -class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable, T] - extends AccumulableParam[R,T] { +private[spark] class +GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] + extends AccumulableParam[R, T] { def addAccumulator(growable: R, elem: T): R = { growable += elem diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9d7c2c8d3d630..c639b3e15ded5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -756,7 +756,7 @@ class SparkContext(config: SparkConf) extends Logging { * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by * standard mutable collections. So you can use this with mutable Map, Set, etc. */ - def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T] + def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] (initialValue: R): Accumulable[R, T] = { val param = new GrowableAccumulableParam[R,T] new Accumulable(initialValue, param) @@ -767,7 +767,7 @@ class SparkContext(config: SparkConf) extends Logging { * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T): Broadcast[T] = { + def broadcast[T: ClassTag](value: T): Broadcast[T] = { val bc = env.broadcastManager.newBroadcast[T](value, isLocal) cleaner.foreach(_.registerBroadcastForCleanup(bc)) bc diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 8b95cda511643..a7cfee6d01711 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -447,7 +447,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value) + def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value)(fakeClassTag) /** Shut down the SparkContext. */ def stop() { diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 738a3b1bed7f3..76956f6a345d1 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -21,6 +21,8 @@ import java.io.Serializable import org.apache.spark.SparkException +import scala.reflect.ClassTag + /** * A broadcast variable. Broadcast variables allow the programmer to keep a read-only variable * cached on each machine rather than shipping a copy of it with tasks. They can be used, for @@ -50,7 +52,7 @@ import org.apache.spark.SparkException * @param id A unique identifier for the broadcast variable. * @tparam T Type of the data contained in the broadcast variable. */ -abstract class Broadcast[T](val id: Long) extends Serializable { +abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { /** * Flag signifying whether the broadcast variable is valid diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 8c8ce9b1691ac..a8c827030a1ef 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import scala.reflect.ClassTag + import org.apache.spark.SecurityManager import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi @@ -31,7 +33,7 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi trait BroadcastFactory { def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit - def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] + def newBroadcast[T: ClassTag](value: T, isLocal: Boolean, id: Long): Broadcast[T] def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala index cf62aca4d45e8..c88be6aba6901 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala @@ -19,6 +19,8 @@ package org.apache.spark.broadcast import java.util.concurrent.atomic.AtomicLong +import scala.reflect.ClassTag + import org.apache.spark._ private[spark] class BroadcastManager( @@ -56,7 +58,7 @@ private[spark] class BroadcastManager( private val nextBroadcastId = new AtomicLong(0) - def newBroadcast[T](value_ : T, isLocal: Boolean) = { + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean) = { broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 29372f16f2cac..78fc286e5192c 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -22,6 +22,8 @@ import java.io.{BufferedInputStream, BufferedOutputStream} import java.net.{URL, URLConnection, URI} import java.util.concurrent.TimeUnit +import scala.reflect.ClassTag + import org.apache.spark.{HttpServer, Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} @@ -34,7 +36,8 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH * (through a HTTP server running at the driver) and stored in the BlockManager of the * executor to speed up future accesses. */ -private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) +private[spark] class HttpBroadcast[T: ClassTag]( + @transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { def getValue = value_ @@ -173,7 +176,7 @@ private[spark] object HttpBroadcast extends Logging { files += file.getAbsolutePath } - def read[T](id: Long): T = { + def read[T: ClassTag](id: Long): T = { logDebug("broadcast read server: " + serverUri + " id: broadcast-" + id) val url = serverUri + "/" + BroadcastBlockId(id).name diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala index e3f6cdc6154dd..d5a031e2bbb59 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import scala.reflect.ClassTag + import org.apache.spark.{SecurityManager, SparkConf} /** @@ -29,7 +31,7 @@ class HttpBroadcastFactory extends BroadcastFactory { HttpBroadcast.initialize(isDriver, conf, securityMgr) } - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = new HttpBroadcast[T](value_, isLocal, id) def stop() { HttpBroadcast.stop() } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 2659274c5e98e..734de37ba115d 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -19,6 +19,7 @@ package org.apache.spark.broadcast import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} +import scala.reflect.ClassTag import scala.math import scala.util.Random @@ -44,7 +45,8 @@ import org.apache.spark.util.Utils * copies of the broadcast data (one per executor) as done by the * [[org.apache.spark.broadcast.HttpBroadcast]]. */ -private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) +private[spark] class TorrentBroadcast[T: ClassTag]( + @transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { def getValue = value_ diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala index d216b58718148..1de8396a0e17f 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala @@ -17,6 +17,8 @@ package org.apache.spark.broadcast +import scala.reflect.ClassTag + import org.apache.spark.{SecurityManager, SparkConf} /** @@ -30,7 +32,7 @@ class TorrentBroadcastFactory extends BroadcastFactory { TorrentBroadcast.initialize(isDriver, conf) } - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = new TorrentBroadcast[T](value_, isLocal, id) def stop() { TorrentBroadcast.stop() } diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 888af541cf970..34c51b833025e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -84,7 +84,7 @@ private[spark] object CheckpointRDD extends Logging { "part-%05d".format(splitId) } - def writeToFile[T]( + def writeToFile[T: ClassTag]( path: String, broadcastedConf: Broadcast[SerializableWritable[Configuration]], blockSize: Int = -1 @@ -160,7 +160,7 @@ private[spark] object CheckpointRDD extends Logging { val conf = SparkHadoopUtil.get.newConfiguration() val fs = path.getFileSystem(conf) val broadcastedConf = sc.broadcast(new SerializableWritable(conf)) - sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf, 1024) _) + sc.runJob(rdd, CheckpointRDD.writeToFile[Int](path.toString, broadcastedConf, 1024) _) val cpRDD = new CheckpointRDD[Int](sc, path.toString) assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same") assert(cpRDD.collect.toList == rdd.collect.toList, "Data of partitions not the same") diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 5f03d7d650a30..2425929fc73c5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -77,7 +77,7 @@ private[spark] class ParallelCollectionPartition[T: ClassTag]( slice = in.readInt() val ser = sfactory.newInstance() - Utils.deserializeViaNestedStream(in, ser)(ds => values = ds.readObject()) + Utils.deserializeViaNestedStream(in, ser)(ds => values = ds.readObject[Seq[T]]()) } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 953f0555e57c5..c3b2a33fb54d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -92,7 +92,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) // Save to file, and reload it as an RDD val broadcastedConf = rdd.context.broadcast( new SerializableWritable(rdd.context.hadoopConfiguration)) - rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf) _) + rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) if (newRDD.partitions.size != rdd.partitions.size) { throw new SparkException( diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index e9163deaf2036..0a7e1ec539679 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -20,6 +20,8 @@ package org.apache.spark.serializer import java.io._ import java.nio.ByteBuffer +import scala.reflect.ClassTag + import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream @@ -36,7 +38,7 @@ private[spark] class JavaSerializationStream(out: OutputStream, counterReset: In * But only call it every 10,000th time to avoid bloated serialization streams (when * the stream 'resets' object class descriptions have to be re-written) */ - def writeObject[T](t: T): SerializationStream = { + def writeObject[T: ClassTag](t: T): SerializationStream = { objOut.writeObject(t) if (counterReset > 0 && counter >= counterReset) { objOut.reset() @@ -46,6 +48,7 @@ private[spark] class JavaSerializationStream(out: OutputStream, counterReset: In } this } + def flush() { objOut.flush() } def close() { objOut.close() } } @@ -57,12 +60,12 @@ extends DeserializationStream { Class.forName(desc.getName, false, loader) } - def readObject[T](): T = objIn.readObject().asInstanceOf[T] + def readObject[T: ClassTag](): T = objIn.readObject().asInstanceOf[T] def close() { objIn.close() } } private[spark] class JavaSerializerInstance(counterReset: Int) extends SerializerInstance { - def serialize[T](t: T): ByteBuffer = { + def serialize[T: ClassTag](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) out.writeObject(t) @@ -70,13 +73,13 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize ByteBuffer.wrap(bos.toByteArray) } - def deserialize[T](bytes: ByteBuffer): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis) in.readObject().asInstanceOf[T] } - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis, loader) in.readObject().asInstanceOf[T] diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index c4daec7875d26..5286f7b4c211a 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -31,6 +31,8 @@ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} +import scala.reflect.ClassTag + /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. * @@ -95,7 +97,7 @@ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) - def writeObject[T](t: T): SerializationStream = { + def writeObject[T: ClassTag](t: T): SerializationStream = { kryo.writeClassAndObject(output, t) this } @@ -108,7 +110,7 @@ private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { val input = new KryoInput(inStream) - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { try { kryo.readClassAndObject(input).asInstanceOf[T] } catch { @@ -131,18 +133,18 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ lazy val output = ks.newKryoOutput() lazy val input = new KryoInput() - def serialize[T](t: T): ByteBuffer = { + def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() kryo.writeClassAndObject(output, t) ByteBuffer.wrap(output.toBytes) } - def deserialize[T](bytes: ByteBuffer): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer): T = { input.setBuffer(bytes.array) kryo.readClassAndObject(input).asInstanceOf[T] } - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val oldClassLoader = kryo.getClassLoader kryo.setClassLoader(loader) input.setBuffer(bytes.array) diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index f2c8f9b6218d6..ee26970a3d874 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -20,6 +20,8 @@ package org.apache.spark.serializer import java.io.{ByteArrayOutputStream, EOFException, InputStream, OutputStream} import java.nio.ByteBuffer +import scala.reflect.ClassTag + import org.apache.spark.SparkEnv import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.{ByteBufferInputStream, NextIterator} @@ -59,17 +61,17 @@ object Serializer { */ @DeveloperApi trait SerializerInstance { - def serialize[T](t: T): ByteBuffer + def serialize[T: ClassTag](t: T): ByteBuffer - def deserialize[T](bytes: ByteBuffer): T + def deserialize[T: ClassTag](bytes: ByteBuffer): T - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T def serializeStream(s: OutputStream): SerializationStream def deserializeStream(s: InputStream): DeserializationStream - def serializeMany[T](iterator: Iterator[T]): ByteBuffer = { + def serializeMany[T: ClassTag](iterator: Iterator[T]): ByteBuffer = { // Default implementation uses serializeStream val stream = new ByteArrayOutputStream() serializeStream(stream).writeAll(iterator) @@ -85,18 +87,17 @@ trait SerializerInstance { } } - /** * :: DeveloperApi :: * A stream for writing serialized objects. */ @DeveloperApi trait SerializationStream { - def writeObject[T](t: T): SerializationStream + def writeObject[T: ClassTag](t: T): SerializationStream def flush(): Unit def close(): Unit - def writeAll[T](iter: Iterator[T]): SerializationStream = { + def writeAll[T: ClassTag](iter: Iterator[T]): SerializationStream = { while (iter.hasNext) { writeObject(iter.next()) } @@ -111,7 +112,7 @@ trait SerializationStream { */ @DeveloperApi trait DeserializationStream { - def readObject[T](): T + def readObject[T: ClassTag](): T def close(): Unit /** diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 3f0ed61c5bbfb..95777fbf57d8b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -850,7 +850,7 @@ private[spark] object Utils extends Logging { /** * Clone an object using a Spark serializer. */ - def clone[T](value: T, serializer: SerializerInstance): T = { + def clone[T: ClassTag](value: T, serializer: SerializerInstance): T = { serializer.deserialize[T](serializer.serialize(value)) } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 5d4673aebe9e8..cdd6b3d8feed7 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.serializer import scala.collection.mutable +import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite @@ -31,7 +32,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("basic types") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check(1) @@ -61,7 +62,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("pairs") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check((1, 1)) @@ -85,7 +86,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("Scala data structures") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } check(List[Int]()) @@ -108,7 +109,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("ranges") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) // Check that very long ranges don't get written one element at a time assert(ser.serialize(t).limit < 100) @@ -129,7 +130,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("custom registrator") { val ser = new KryoSerializer(conf).newInstance() - def check[T](t: T) { + def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index a197dac87d6db..576a3e371b993 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -28,6 +28,8 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag + object WikipediaPageRankStandalone { def main(args: Array[String]) { if (args.length < 4) { @@ -143,15 +145,15 @@ class WPRSerializer extends org.apache.spark.serializer.Serializer { } class WPRSerializerInstance extends SerializerInstance { - def serialize[T](t: T): ByteBuffer = { + def serialize[T: ClassTag](t: T): ByteBuffer = { throw new UnsupportedOperationException() } - def deserialize[T](bytes: ByteBuffer): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer): T = { throw new UnsupportedOperationException() } - def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = { + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { throw new UnsupportedOperationException() } @@ -167,7 +169,7 @@ class WPRSerializerInstance extends SerializerInstance { class WPRSerializationStream(os: OutputStream) extends SerializationStream { val dos = new DataOutputStream(os) - def writeObject[T](t: T): SerializationStream = t match { + def writeObject[T: ClassTag](t: T): SerializationStream = t match { case (id: String, wrapper: ArrayBuffer[_]) => wrapper(0) match { case links: Array[String] => { dos.writeInt(0) // links @@ -200,7 +202,7 @@ class WPRSerializationStream(os: OutputStream) extends SerializationStream { class WPRDeserializationStream(is: InputStream) extends DeserializationStream { val dis = new DataInputStream(is) - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { val typeId = dis.readInt() typeId match { case 0 => { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 2f0531ee5f379..1de42eeca1f00 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -17,20 +17,22 @@ package org.apache.spark.graphx.impl +import scala.language.existentials + import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer +import scala.reflect.ClassTag + import org.apache.spark.graphx._ import org.apache.spark.serializer._ -import scala.language.existentials - private[graphx] class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[(VertexId, _)] writeVarLong(msg._1, optimizePositive = false) this @@ -38,7 +40,7 @@ class VertexIdMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { (readVarLong(optimizePositive = false), null).asInstanceOf[T] } } @@ -51,7 +53,7 @@ class IntVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Int]] writeVarLong(msg.vid, optimizePositive = false) writeInt(msg.data) @@ -60,7 +62,7 @@ class IntVertexBroadcastMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readInt() new VertexBroadcastMsg[Int](0, a, b).asInstanceOf[T] @@ -75,7 +77,7 @@ class LongVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Long]] writeVarLong(msg.vid, optimizePositive = false) writeLong(msg.data) @@ -84,7 +86,7 @@ class LongVertexBroadcastMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readLong() new VertexBroadcastMsg[Long](0, a, b).asInstanceOf[T] @@ -99,7 +101,7 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[VertexBroadcastMsg[Double]] writeVarLong(msg.vid, optimizePositive = false) writeDouble(msg.data) @@ -108,7 +110,7 @@ class DoubleVertexBroadcastMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readDouble() new VertexBroadcastMsg[Double](0, a, b).asInstanceOf[T] @@ -123,7 +125,7 @@ class IntAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[(VertexId, Int)] writeVarLong(msg._1, optimizePositive = false) writeUnsignedVarInt(msg._2) @@ -132,7 +134,7 @@ class IntAggMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readUnsignedVarInt() (a, b).asInstanceOf[T] @@ -147,7 +149,7 @@ class LongAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[(VertexId, Long)] writeVarLong(msg._1, optimizePositive = false) writeVarLong(msg._2, optimizePositive = true) @@ -156,7 +158,7 @@ class LongAggMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T](): T = { + override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readVarLong(optimizePositive = true) (a, b).asInstanceOf[T] @@ -171,7 +173,7 @@ class DoubleAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T](t: T) = { + def writeObject[T: ClassTag](t: T) = { val msg = t.asInstanceOf[(VertexId, Double)] writeVarLong(msg._1, optimizePositive = false) writeDouble(msg._2) @@ -180,7 +182,7 @@ class DoubleAggMsgSerializer extends Serializer with Serializable { } override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) val b = readDouble() (a, b).asInstanceOf[T] @@ -196,7 +198,7 @@ class DoubleAggMsgSerializer extends Serializer with Serializable { private[graphx] abstract class ShuffleSerializationStream(s: OutputStream) extends SerializationStream { // The implementation should override this one. - def writeObject[T](t: T): SerializationStream + def writeObject[T: ClassTag](t: T): SerializationStream def writeInt(v: Int) { s.write(v >> 24) @@ -309,7 +311,7 @@ abstract class ShuffleSerializationStream(s: OutputStream) extends Serialization private[graphx] abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream { // The implementation should override this one. - def readObject[T](): T + def readObject[T: ClassTag](): T def readInt(): Int = { val first = s.read() @@ -398,11 +400,12 @@ abstract class ShuffleDeserializationStream(s: InputStream) extends Deserializat private[graphx] sealed trait ShuffleSerializerInstance extends SerializerInstance { - override def serialize[T](t: T): ByteBuffer = throw new UnsupportedOperationException + override def serialize[T: ClassTag](t: T): ByteBuffer = throw new UnsupportedOperationException - override def deserialize[T](bytes: ByteBuffer): T = throw new UnsupportedOperationException + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = + throw new UnsupportedOperationException - override def deserialize[T](bytes: ByteBuffer, loader: ClassLoader): T = + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = throw new UnsupportedOperationException // The implementation should override the following two. diff --git a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala index 73438d9535962..91caa6b605a1e 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream} import scala.util.Random +import scala.reflect.ClassTag import org.scalatest.FunSuite @@ -164,7 +165,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { def testVarLongEncoding(v: Long, optimizePositive: Boolean) { val bout = new ByteArrayOutputStream val stream = new ShuffleSerializationStream(bout) { - def writeObject[T](t: T): SerializationStream = { + def writeObject[T: ClassTag](t: T): SerializationStream = { writeVarLong(t.asInstanceOf[Long], optimizePositive = optimizePositive) this } @@ -173,7 +174,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext { val bin = new ByteArrayInputStream(bout.toByteArray) val dstream = new ShuffleDeserializationStream(bin) { - def readObject[T](): T = { + def readObject[T: ClassTag](): T = { readVarLong(optimizePositive).asInstanceOf[T] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 5067c14ddffeb..1c6e29b3cdee9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution import java.nio.ByteBuffer +import scala.reflect.ClassTag + import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} @@ -59,11 +61,11 @@ private[sql] object SparkSqlSerializer { new KryoSerializer(sparkConf) } - def serialize[T](o: T): Array[Byte] = { + def serialize[T: ClassTag](o: T): Array[Byte] = { ser.newInstance().serialize(o).array() } - def deserialize[T](bytes: Array[Byte]): T = { + def deserialize[T: ClassTag](bytes: Array[Byte]): T = { ser.newInstance().deserialize[T](ByteBuffer.wrap(bytes)) } } From c05d11bb307eaba40c5669da2d374c28debaa55a Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Sat, 10 May 2014 12:46:51 -0700 Subject: [PATCH 07/82] fix broken in link in python docs Author: Andy Konwinski Closes #650 from andyk/python-docs-link-fix and squashes the following commits: a1f9d51 [Andy Konwinski] fix broken in link in python docs --- docs/python-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 6813963bb080c..39fb5f0c99ca3 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -45,7 +45,7 @@ errors = logData.filter(is_error) PySpark will automatically ship these functions to executors, along with any objects that they reference. Instances of classes will be serialized and shipped to executors by PySpark, but classes themselves cannot be automatically distributed to executors. -The [Standalone Use](#standalone-use) section describes how to ship code dependencies to executors. +The [Standalone Use](#standalone-programs) section describes how to ship code dependencies to executors. In addition, PySpark fully supports interactive use---simply run `./bin/pyspark` to launch an interactive shell. From 3776f2f283842543ff766398292532c6e94221cc Mon Sep 17 00:00:00 2001 From: Bouke van der Bijl Date: Sat, 10 May 2014 13:02:13 -0700 Subject: [PATCH 08/82] Add Python includes to path before depickling broadcast values This fixes https://issues.apache.org/jira/browse/SPARK-1731 by adding the Python includes to the PYTHONPATH before depickling the broadcast values @airhorns Author: Bouke van der Bijl Closes #656 from bouk/python-includes-before-broadcast and squashes the following commits: 7b0dfe4 [Bouke van der Bijl] Add Python includes to path before depickling broadcast values --- .../org/apache/spark/api/python/PythonRDD.scala | 10 +++++----- python/pyspark/worker.py | 14 +++++++------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index fecd9762f3f60..388b838d78bba 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -179,6 +179,11 @@ private[spark] class PythonRDD[T: ClassTag]( dataOut.writeInt(split.index) // sparkFilesDir PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut) + // Python includes (*.zip and *.egg files) + dataOut.writeInt(pythonIncludes.length) + for (include <- pythonIncludes) { + PythonRDD.writeUTF(include, dataOut) + } // Broadcast variables dataOut.writeInt(broadcastVars.length) for (broadcast <- broadcastVars) { @@ -186,11 +191,6 @@ private[spark] class PythonRDD[T: ClassTag]( dataOut.writeInt(broadcast.value.length) dataOut.write(broadcast.value) } - // Python includes (*.zip and *.egg files) - dataOut.writeInt(pythonIncludes.length) - for (include <- pythonIncludes) { - PythonRDD.writeUTF(include, dataOut) - } dataOut.flush() // Serialized command: dataOut.writeInt(command.length) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 4c214ef359685..f43210c6c0301 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -56,13 +56,6 @@ def main(infile, outfile): SparkFiles._root_directory = spark_files_dir SparkFiles._is_running_on_worker = True - # fetch names and values of broadcast variables - num_broadcast_variables = read_int(infile) - for _ in range(num_broadcast_variables): - bid = read_long(infile) - value = pickleSer._read_with_length(infile) - _broadcastRegistry[bid] = Broadcast(bid, value) - # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH sys.path.append(spark_files_dir) # *.py files that were added will be copied here num_python_includes = read_int(infile) @@ -70,6 +63,13 @@ def main(infile, outfile): filename = utf8_deserializer.loads(infile) sys.path.append(os.path.join(spark_files_dir, filename)) + # fetch names and values of broadcast variables + num_broadcast_variables = read_int(infile) + for _ in range(num_broadcast_variables): + bid = read_long(infile) + value = pickleSer._read_with_length(infile) + _broadcastRegistry[bid] = Broadcast(bid, value) + command = pickleSer._read_with_length(infile) (func, deserializer, serializer) = command init_time = time.time() From 6c2691d0a0ed46a8b8093e05a4708706cf187168 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Sat, 10 May 2014 14:01:08 -0700 Subject: [PATCH 09/82] [SPARK-1690] Tolerating empty elements when saving Python RDD to text files Tolerate empty strings in PythonRDD Author: Kan Zhang Closes #644 from kanzhang/SPARK-1690 and squashes the following commits: c62ad33 [Kan Zhang] Adding Python doctest 473ec4b [Kan Zhang] [SPARK-1690] Tolerating empty elements when saving Python RDD to text files --- .../scala/org/apache/spark/api/python/PythonRDD.scala | 5 +++-- python/pyspark/rdd.py | 8 ++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 388b838d78bba..2971c277aa863 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -94,6 +94,7 @@ private[spark] class PythonRDD[T: ClassTag]( val obj = new Array[Byte](length) stream.readFully(obj) obj + case 0 => Array.empty[Byte] case SpecialLengths.TIMING_DATA => // Timing data from worker val bootTime = stream.readLong() @@ -123,7 +124,7 @@ private[spark] class PythonRDD[T: ClassTag]( stream.readFully(update) accumulator += Collections.singletonList(update) } - Array.empty[Byte] + null } } catch { @@ -143,7 +144,7 @@ private[spark] class PythonRDD[T: ClassTag]( var _nextObj = read() - def hasNext = _nextObj.length != 0 + def hasNext = _nextObj != null } new InterruptibleIterator(context, stdoutIterator) } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3a1c56af5b221..4f74824ba4cf2 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -891,6 +891,14 @@ def saveAsTextFile(self, path): >>> from glob import glob >>> ''.join(sorted(input(glob(tempFile.name + "/part-0000*")))) '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n' + + Empty lines are tolerated when saving to text files. + + >>> tempFile2 = NamedTemporaryFile(delete=True) + >>> tempFile2.close() + >>> sc.parallelize(['', 'foo', '', 'bar', '']).saveAsTextFile(tempFile2.name) + >>> ''.join(sorted(input(glob(tempFile2.name + "/part-0000*")))) + '\\n\\n\\nbar\\nfoo\\n' """ def func(split, iterator): for x in iterator: From 905173df57b90f90ebafb22e43f55164445330e6 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sat, 10 May 2014 14:48:07 -0700 Subject: [PATCH 10/82] Unify GraphImpl RDDs + other graph load optimizations This PR makes the following changes, primarily in e4fbd329aef85fe2c38b0167255d2a712893d683: 1. *Unify RDDs to avoid zipPartitions.* A graph used to be four RDDs: vertices, edges, routing table, and triplet view. This commit merges them down to two: vertices (with routing table), and edges (with replicated vertices). 2. *Avoid duplicate shuffle in graph building.* We used to do two shuffles when building a graph: one to extract routing information from the edges and move it to the vertices, and another to find nonexistent vertices referred to by edges. With this commit, the latter is done as a side effect of the former. 3. *Avoid no-op shuffle when joins are fully eliminated.* This is a side effect of unifying the edges and the triplet view. 4. *Join elimination for mapTriplets.* 5. *Ship only the needed vertex attributes when upgrading the triplet view.* If the triplet view already contains source attributes, and we now need both attributes, only ship destination attributes rather than re-shipping both. This is done in `ReplicatedVertexView#upgrade`. Author: Ankur Dave Closes #497 from ankurdave/unify-rdds and squashes the following commits: 332ab43 [Ankur Dave] Merge remote-tracking branch 'apache-spark/master' into unify-rdds 4933e2e [Ankur Dave] Exclude RoutingTable from binary compatibility check 5ba8789 [Ankur Dave] Add GraphX upgrade guide from Spark 0.9.1 13ac845 [Ankur Dave] Merge remote-tracking branch 'apache-spark/master' into unify-rdds a04765c [Ankur Dave] Remove unnecessary toOps call 57202e8 [Ankur Dave] Replace case with pair parameter 75af062 [Ankur Dave] Add explicit return types 04d3ae5 [Ankur Dave] Convert implicit parameter to context bound c88b269 [Ankur Dave] Revert upgradeIterator to if-in-a-loop 0d3584c [Ankur Dave] EdgePartition.size should be val 2a928b2 [Ankur Dave] Set locality wait 10b3596 [Ankur Dave] Clean up public API ae36110 [Ankur Dave] Fix style errors e4fbd32 [Ankur Dave] Unify GraphImpl RDDs + other graph load optimizations d6d60e2 [Ankur Dave] In GraphLoader, coalesce to minEdgePartitions 62c7b78 [Ankur Dave] In Analytics, take PageRank numIter d64e8d4 [Ankur Dave] Log current Pregel iteration --- docs/graphx-programming-guide.md | 22 +- .../org/apache/spark/graphx/EdgeRDD.scala | 56 +-- .../org/apache/spark/graphx/EdgeTriplet.scala | 2 + .../scala/org/apache/spark/graphx/Graph.scala | 2 +- .../spark/graphx/GraphKryoRegistrator.scala | 8 +- .../org/apache/spark/graphx/GraphLoader.scala | 10 +- .../org/apache/spark/graphx/GraphOps.scala | 17 +- .../org/apache/spark/graphx/Pregel.scala | 6 +- .../org/apache/spark/graphx/VertexRDD.scala | 166 ++++++--- .../spark/graphx/impl/EdgePartition.scala | 132 +++++-- .../graphx/impl/EdgePartitionBuilder.scala | 18 +- .../graphx/impl/EdgeTripletIterator.scala | 50 ++- .../apache/spark/graphx/impl/GraphImpl.scala | 344 +++++++----------- .../graphx/impl/MessageToPartition.scala | 21 +- .../graphx/impl/ReplicatedVertexView.scala | 238 ++++-------- .../spark/graphx/impl/RoutingTable.scala | 82 ----- .../graphx/impl/RoutingTablePartition.scala | 158 ++++++++ .../spark/graphx/impl/Serializers.scala | 29 ++ .../impl/ShippableVertexPartition.scala | 149 ++++++++ .../spark/graphx/impl/VertexPartition.scala | 269 ++------------ .../graphx/impl/VertexPartitionBase.scala | 91 +++++ .../graphx/impl/VertexPartitionBaseOps.scala | 245 +++++++++++++ .../apache/spark/graphx/lib/Analytics.scala | 8 +- .../org/apache/spark/graphx/GraphSuite.scala | 10 +- .../graphx/impl/EdgePartitionSuite.scala | 48 ++- .../impl/EdgeTripletIteratorSuite.scala | 10 +- .../graphx/impl/VertexPartitionSuite.scala | 11 - project/MimaBuild.scala | 2 + 28 files changed, 1353 insertions(+), 851 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 07be8ba58efa3..42ab27bf55ccf 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -86,6 +86,12 @@ support the [Bagel API](api/scala/index.html#org.apache.spark.bagel.package) and [Bagel programming guide](bagel-programming-guide.html). However, we encourage Bagel users to explore the new GraphX API and comment on issues that may complicate the transition from Bagel. +## Upgrade Guide from Spark 0.9.1 + +GraphX in Spark {{site.SPARK_VERSION}} contains one user-facing interface change from Spark 0.9.1. [`EdgeRDD`][EdgeRDD] may now store adjacent vertex attributes to construct the triplets, so it has gained a type parameter. The edges of a graph of type `Graph[VD, ED]` are of type `EdgeRDD[ED, VD]` rather than `EdgeRDD[ED]`. + +[EdgeRDD]: api/scala/index.html#org.apache.spark.graphx.EdgeRDD + # Getting Started To get started you first need to import Spark and GraphX into your project, as follows: @@ -145,12 +151,12 @@ the vertices and edges of the graph: {% highlight scala %} class Graph[VD, ED] { val vertices: VertexRDD[VD] - val edges: EdgeRDD[ED] + val edges: EdgeRDD[ED, VD] } {% endhighlight %} -The classes `VertexRDD[VD]` and `EdgeRDD[ED]` extend and are optimized versions of `RDD[(VertexID, -VD)]` and `RDD[Edge[ED]]` respectively. Both `VertexRDD[VD]` and `EdgeRDD[ED]` provide additional +The classes `VertexRDD[VD]` and `EdgeRDD[ED, VD]` extend and are optimized versions of `RDD[(VertexID, +VD)]` and `RDD[Edge[ED]]` respectively. Both `VertexRDD[VD]` and `EdgeRDD[ED, VD]` provide additional functionality built around graph computation and leverage internal optimizations. We discuss the `VertexRDD` and `EdgeRDD` API in greater detail in the section on [vertex and edge RDDs](#vertex_and_edge_rdds) but for now they can be thought of as simply RDDs of the form: @@ -302,7 +308,7 @@ class Graph[VD, ED] { val degrees: VertexRDD[Int] // Views of the graph as collections ============================================================= val vertices: VertexRDD[VD] - val edges: EdgeRDD[ED] + val edges: EdgeRDD[ED, VD] val triplets: RDD[EdgeTriplet[VD, ED]] // Functions for caching graphs ================================================================== def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] @@ -908,7 +914,7 @@ val setC: VertexRDD[Double] = setA.innerJoin(setB)((id, a, b) => a + b) ## EdgeRDDs -The `EdgeRDD[ED]`, which extends `RDD[Edge[ED]]` organizes the edges in blocks partitioned using one +The `EdgeRDD[ED, VD]`, which extends `RDD[Edge[ED]]` organizes the edges in blocks partitioned using one of the various partitioning strategies defined in [`PartitionStrategy`][PartitionStrategy]. Within each partition, edge attributes and adjacency structure, are stored separately enabling maximum reuse when changing attribute values. @@ -918,11 +924,11 @@ reuse when changing attribute values. The three additional functions exposed by the `EdgeRDD` are: {% highlight scala %} // Transform the edge attributes while preserving the structure -def mapValues[ED2](f: Edge[ED] => ED2): EdgeRDD[ED2] +def mapValues[ED2](f: Edge[ED] => ED2): EdgeRDD[ED2, VD] // Revere the edges reusing both attributes and structure -def reverse: EdgeRDD[ED] +def reverse: EdgeRDD[ED, VD] // Join two `EdgeRDD`s partitioned using the same partitioning strategy. -def innerJoin[ED2, ED3](other: EdgeRDD[ED2])(f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3] +def innerJoin[ED2, ED3](other: EdgeRDD[ED2, VD])(f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] {% endhighlight %} In most applications we have found that operations on the `EdgeRDD` are accomplished through the diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index fa78ca99b8891..a8fc095072512 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -20,16 +20,19 @@ package org.apache.spark.graphx import scala.reflect.{classTag, ClassTag} import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} -import org.apache.spark.graphx.impl.EdgePartition import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.graphx.impl.EdgePartition + /** - * `EdgeRDD[ED]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each partition - * for performance. + * `EdgeRDD[ED, VD]` extends `RDD[Edge[ED]]` by storing the edges in columnar format on each + * partition for performance. It may additionally store the vertex attributes associated with each + * edge to provide the triplet view. Shipping of the vertex attributes is managed by + * `impl.ReplicatedVertexView`. */ -class EdgeRDD[@specialized ED: ClassTag]( - val partitionsRDD: RDD[(PartitionID, EdgePartition[ED])]) +class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( + val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])]) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { partitionsRDD.setName("EdgeRDD") @@ -45,8 +48,12 @@ class EdgeRDD[@specialized ED: ClassTag]( partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { - val p = firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context) - p.next._2.iterator.map(_.copy()) + val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context) + if (p.hasNext) { + p.next._2.iterator.map(_.copy()) + } else { + Iterator.empty + } } override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() @@ -61,11 +68,15 @@ class EdgeRDD[@specialized ED: ClassTag]( this } - private[graphx] def mapEdgePartitions[ED2: ClassTag]( - f: (PartitionID, EdgePartition[ED]) => EdgePartition[ED2]): EdgeRDD[ED2] = { - new EdgeRDD[ED2](partitionsRDD.mapPartitions({ iter => - val (pid, ep) = iter.next() - Iterator(Tuple2(pid, f(pid, ep))) + private[graphx] def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag]( + f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDD[ED2, VD2] = { + new EdgeRDD[ED2, VD2](partitionsRDD.mapPartitions({ iter => + if (iter.hasNext) { + val (pid, ep) = iter.next() + Iterator(Tuple2(pid, f(pid, ep))) + } else { + Iterator.empty + } }, preservesPartitioning = true)) } @@ -76,7 +87,7 @@ class EdgeRDD[@specialized ED: ClassTag]( * @param f the function from an edge to a new edge value * @return a new EdgeRDD containing the new edge values */ - def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2] = + def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDD[ED2, VD] = mapEdgePartitions((pid, part) => part.map(f)) /** @@ -84,7 +95,14 @@ class EdgeRDD[@specialized ED: ClassTag]( * * @return a new EdgeRDD containing all the edges reversed */ - def reverse: EdgeRDD[ED] = mapEdgePartitions((pid, part) => part.reverse) + def reverse: EdgeRDD[ED, VD] = mapEdgePartitions((pid, part) => part.reverse) + + /** Removes all edges but those matching `epred` and where both vertices match `vpred`. */ + def filter( + epred: EdgeTriplet[VD, ED] => Boolean, + vpred: (VertexId, VD) => Boolean): EdgeRDD[ED, VD] = { + mapEdgePartitions((pid, part) => part.filter(epred, vpred)) + } /** * Inner joins this EdgeRDD with another EdgeRDD, assuming both are partitioned using the same @@ -96,19 +114,15 @@ class EdgeRDD[@specialized ED: ClassTag]( * with values supplied by `f` */ def innerJoin[ED2: ClassTag, ED3: ClassTag] - (other: EdgeRDD[ED2]) - (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3] = { + (other: EdgeRDD[ED2, _]) + (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3, VD] = { val ed2Tag = classTag[ED2] val ed3Tag = classTag[ED3] - new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, true) { + new EdgeRDD[ED3, VD](partitionsRDD.zipPartitions(other.partitionsRDD, true) { (thisIter, otherIter) => val (pid, thisEPart) = thisIter.next() val (_, otherEPart) = otherIter.next() Iterator(Tuple2(pid, thisEPart.innerJoin(otherEPart)(f)(ed2Tag, ed3Tag))) }) } - - private[graphx] def collectVertexIds(): RDD[VertexId] = { - partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) } - } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala index dfc6a801587d2..9d473d5ebda44 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -63,4 +63,6 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr } override def toString = ((srcId, srcAttr), (dstId, dstAttr), attr).toString() + + def toTuple: ((VertexId, VD), (VertexId, VD), ED) = ((srcId, srcAttr), (dstId, dstAttr), attr) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 50395868902dc..dc5dac4fdad57 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -59,7 +59,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * along with their vertex data. * */ - @transient val edges: EdgeRDD[ED] + @transient val edges: EdgeRDD[ED, VD] /** * An RDD containing the edge triplets, which are edges along with the vertex data associated with diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index dd380d8c182c9..d295d0127ac72 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -19,10 +19,11 @@ package org.apache.spark.graphx import com.esotericsoftware.kryo.Kryo -import org.apache.spark.graphx.impl._ import org.apache.spark.serializer.KryoRegistrator -import org.apache.spark.util.collection.BitSet import org.apache.spark.util.BoundedPriorityQueue +import org.apache.spark.util.collection.BitSet + +import org.apache.spark.graphx.impl._ /** * Registers GraphX classes with Kryo for improved performance. @@ -33,8 +34,9 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[Edge[Object]]) kryo.register(classOf[MessageToPartition[Object]]) kryo.register(classOf[VertexBroadcastMsg[Object]]) + kryo.register(classOf[RoutingTableMessage]) kryo.register(classOf[(VertexId, Object)]) - kryo.register(classOf[EdgePartition[Object]]) + kryo.register(classOf[EdgePartition[Object, Object]]) kryo.register(classOf[BitSet]) kryo.register(classOf[VertexIdToIndexMap]) kryo.register(classOf[VertexAttributeBlock[Object]]) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index 18858466db27b..389490c139848 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -47,8 +47,7 @@ object GraphLoader extends Logging { * @param path the path to the file (e.g., /home/data/file or hdfs://file) * @param canonicalOrientation whether to orient edges in the positive * direction - * @param minEdgePartitions the number of partitions for the - * the edge RDD + * @param minEdgePartitions the number of partitions for the edge RDD */ def edgeListFile( sc: SparkContext, @@ -60,8 +59,9 @@ object GraphLoader extends Logging { val startTime = System.currentTimeMillis // Parse the edge data table directly into edge partitions - val edges = sc.textFile(path, minEdgePartitions).mapPartitionsWithIndex { (pid, iter) => - val builder = new EdgePartitionBuilder[Int] + val lines = sc.textFile(path, minEdgePartitions).coalesce(minEdgePartitions) + val edges = lines.mapPartitionsWithIndex { (pid, iter) => + val builder = new EdgePartitionBuilder[Int, Int] iter.foreach { line => if (!line.isEmpty && line(0) != '#') { val lineArray = line.split("\\s+") @@ -78,7 +78,7 @@ object GraphLoader extends Logging { } } Iterator((pid, builder.toEdgePartition)) - }.cache() + }.cache().setName("GraphLoader.edgeListFile - edges (%s)".format(path)) edges.count() logInfo("It took %d ms to load the edges".format(System.currentTimeMillis - startTime)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 4997fbc3cbcd8..edd5b79da1522 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -18,11 +18,13 @@ package org.apache.spark.graphx import scala.reflect.ClassTag -import org.apache.spark.SparkContext._ +import scala.util.Random + import org.apache.spark.SparkException -import org.apache.spark.graphx.lib._ +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import scala.util.Random + +import org.apache.spark.graphx.lib._ /** * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the @@ -43,19 +45,22 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * The in-degree of each vertex in the graph. * @note Vertices with no in-edges are not returned in the resulting RDD. */ - @transient lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) + @transient lazy val inDegrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.In).setName("GraphOps.inDegrees") /** * The out-degree of each vertex in the graph. * @note Vertices with no out-edges are not returned in the resulting RDD. */ - @transient lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) + @transient lazy val outDegrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.Out).setName("GraphOps.outDegrees") /** * The degree of each vertex in the graph. * @note Vertices with no edges are not returned in the resulting RDD. */ - @transient lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) + @transient lazy val degrees: VertexRDD[Int] = + degreesRDD(EdgeDirection.Either).setName("GraphOps.degrees") /** * Computes the neighboring vertex degrees. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index ac07a594a12e4..4572eab2875bb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -18,6 +18,7 @@ package org.apache.spark.graphx import scala.reflect.ClassTag +import org.apache.spark.Logging /** @@ -52,7 +53,7 @@ import scala.reflect.ClassTag * }}} * */ -object Pregel { +object Pregel extends Logging { /** * Execute a Pregel-like iterative vertex-parallel abstraction. The @@ -142,6 +143,9 @@ object Pregel { // hides oldMessages (depended on by newVerts), newVerts (depended on by messages), and the // vertices of prevG (depended on by newVerts, oldMessages, and the vertices of g). activeMessages = messages.count() + + logInfo("Pregel finished iteration " + i) + // Unpersist the RDDs hidden by newly-materialized RDDs oldMessages.unpersist(blocking=false) newVerts.unpersist(blocking=false) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index f0fc605c88575..8c62897037b6d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -24,8 +24,11 @@ import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.graphx.impl.MsgRDDFunctions -import org.apache.spark.graphx.impl.VertexPartition +import org.apache.spark.graphx.impl.RoutingTablePartition +import org.apache.spark.graphx.impl.ShippableVertexPartition +import org.apache.spark.graphx.impl.VertexAttributeBlock +import org.apache.spark.graphx.impl.RoutingTableMessageRDDFunctions._ +import org.apache.spark.graphx.impl.VertexRDDFunctions._ /** * Extends `RDD[(VertexId, VD)]` by ensuring that there is only one entry for each vertex and by @@ -33,6 +36,9 @@ import org.apache.spark.graphx.impl.VertexPartition * joined efficiently. All operations except [[reindex]] preserve the index. To construct a * `VertexRDD`, use the [[org.apache.spark.graphx.VertexRDD$ VertexRDD object]]. * + * Additionally, stores routing information to enable joining the vertex attributes with an + * [[EdgeRDD]]. + * * @example Construct a `VertexRDD` from a plain RDD: * {{{ * // Construct an initial vertex set @@ -50,13 +56,11 @@ import org.apache.spark.graphx.impl.VertexPartition * @tparam VD the vertex attribute associated with each vertex in the set. */ class VertexRDD[@specialized VD: ClassTag]( - val partitionsRDD: RDD[VertexPartition[VD]]) + val partitionsRDD: RDD[ShippableVertexPartition[VD]]) extends RDD[(VertexId, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { require(partitionsRDD.partitioner.isDefined) - partitionsRDD.setName("VertexRDD") - /** * Construct a new VertexRDD that is indexed by only the visible vertices. The resulting * VertexRDD will be based on a different index and can no longer be quickly joined with this @@ -71,6 +75,16 @@ class VertexRDD[@specialized VD: ClassTag]( override protected def getPreferredLocations(s: Partition): Seq[String] = partitionsRDD.preferredLocations(s) + override def setName(_name: String): this.type = { + if (partitionsRDD.name != null) { + partitionsRDD.setName(partitionsRDD.name + ", " + _name) + } else { + partitionsRDD.setName(_name) + } + this + } + setName("VertexRDD") + override def persist(newLevel: StorageLevel): this.type = { partitionsRDD.persist(newLevel) this @@ -90,14 +104,14 @@ class VertexRDD[@specialized VD: ClassTag]( * Provides the `RDD[(VertexId, VD)]` equivalent output. */ override def compute(part: Partition, context: TaskContext): Iterator[(VertexId, VD)] = { - firstParent[VertexPartition[VD]].iterator(part, context).next.iterator + firstParent[ShippableVertexPartition[VD]].iterator(part, context).next.iterator } /** * Applies a function to each `VertexPartition` of this RDD and returns a new VertexRDD. */ private[graphx] def mapVertexPartitions[VD2: ClassTag]( - f: VertexPartition[VD] => VertexPartition[VD2]) + f: ShippableVertexPartition[VD] => ShippableVertexPartition[VD2]) : VertexRDD[VD2] = { val newPartitionsRDD = partitionsRDD.mapPartitions(_.map(f), preservesPartitioning = true) new VertexRDD(newPartitionsRDD) @@ -208,10 +222,8 @@ class VertexRDD[@specialized VD: ClassTag]( case _ => new VertexRDD[VD3]( partitionsRDD.zipPartitions( - other.partitionBy(this.partitioner.get), preservesPartitioning = true) - { (part, msgs) => - val vertexPartition: VertexPartition[VD] = part.next() - Iterator(vertexPartition.leftJoin(msgs)(f)) + other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.leftJoin(msgs)(f)) } ) } @@ -254,10 +266,8 @@ class VertexRDD[@specialized VD: ClassTag]( case _ => new VertexRDD( partitionsRDD.zipPartitions( - other.partitionBy(this.partitioner.get), preservesPartitioning = true) - { (part, msgs) => - val vertexPartition: VertexPartition[VD] = part.next() - Iterator(vertexPartition.innerJoin(msgs)(f)) + other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { + (partIter, msgs) => partIter.map(_.innerJoin(msgs)(f)) } ) } @@ -276,14 +286,31 @@ class VertexRDD[@specialized VD: ClassTag]( */ def aggregateUsingIndex[VD2: ClassTag]( messages: RDD[(VertexId, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { - val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get) + val shuffled = messages.copartitionWithVertices(this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => - val vertexPartition: VertexPartition[VD] = thisIter.next() - Iterator(vertexPartition.aggregateUsingIndex(msgIter, reduceFunc)) + thisIter.map(_.aggregateUsingIndex(msgIter, reduceFunc)) } new VertexRDD[VD2](parts) } + /** + * Returns a new `VertexRDD` reflecting a reversal of all edge directions in the corresponding + * [[EdgeRDD]]. + */ + def reverseRoutingTables(): VertexRDD[VD] = + this.mapVertexPartitions(vPart => vPart.withRoutingTable(vPart.routingTable.reverse)) + + /** Generates an RDD of vertex attributes suitable for shipping to the edge partitions. */ + private[graphx] def shipVertexAttributes( + shipSrc: Boolean, shipDst: Boolean): RDD[(PartitionID, VertexAttributeBlock[VD])] = { + partitionsRDD.mapPartitions(_.flatMap(_.shipVertexAttributes(shipSrc, shipDst))) + } + + /** Generates an RDD of vertex IDs suitable for shipping to the edge partitions. */ + private[graphx] def shipVertexIds(): RDD[(PartitionID, Array[VertexId])] = { + partitionsRDD.mapPartitions(_.flatMap(_.shipVertexIds())) + } + } // end of VertexRDD @@ -293,52 +320,101 @@ class VertexRDD[@specialized VD: ClassTag]( object VertexRDD { /** - * Construct a `VertexRDD` from an RDD of vertex-attribute pairs. - * Duplicate entries are removed arbitrarily. + * Constructs a standalone `VertexRDD` (one that is not set up for efficient joins with an + * [[EdgeRDD]]) from an RDD of vertex-attribute pairs. Duplicate entries are removed arbitrarily. * * @tparam VD the vertex attribute type * - * @param rdd the collection of vertex-attribute pairs + * @param vertices the collection of vertex-attribute pairs */ - def apply[VD: ClassTag](rdd: RDD[(VertexId, VD)]): VertexRDD[VD] = { - val partitioned: RDD[(VertexId, VD)] = rdd.partitioner match { - case Some(p) => rdd - case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) + def apply[VD: ClassTag](vertices: RDD[(VertexId, VD)]): VertexRDD[VD] = { + val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { + case Some(p) => vertices + case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size)) } - val vertexPartitions = partitioned.mapPartitions( - iter => Iterator(VertexPartition(iter)), + val vertexPartitions = vPartitioned.mapPartitions( + iter => Iterator(ShippableVertexPartition(iter)), preservesPartitioning = true) new VertexRDD(vertexPartitions) } /** - * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs, merging duplicates using - * `mergeFunc`. + * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs. Duplicate vertex entries are + * removed arbitrarily. The resulting `VertexRDD` will be joinable with `edges`, and any missing + * vertices referred to by `edges` will be created with the attribute `defaultVal`. * * @tparam VD the vertex attribute type * - * @param rdd the collection of vertex-attribute pairs - * @param mergeFunc the associative, commutative merge function. + * @param vertices the collection of vertex-attribute pairs + * @param edges the [[EdgeRDD]] that these vertices may be joined with + * @param defaultVal the vertex attribute to use when creating missing vertices */ - def apply[VD: ClassTag](rdd: RDD[(VertexId, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = { - val partitioned: RDD[(VertexId, VD)] = rdd.partitioner match { - case Some(p) => rdd - case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size)) + def apply[VD: ClassTag]( + vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD): VertexRDD[VD] = { + VertexRDD(vertices, edges, defaultVal, (a, b) => b) + } + + /** + * Constructs a `VertexRDD` from an RDD of vertex-attribute pairs. Duplicate vertex entries are + * merged using `mergeFunc`. The resulting `VertexRDD` will be joinable with `edges`, and any + * missing vertices referred to by `edges` will be created with the attribute `defaultVal`. + * + * @tparam VD the vertex attribute type + * + * @param vertices the collection of vertex-attribute pairs + * @param edges the [[EdgeRDD]] that these vertices may be joined with + * @param defaultVal the vertex attribute to use when creating missing vertices + * @param mergeFunc the commutative, associative duplicate vertex attribute merge function + */ + def apply[VD: ClassTag]( + vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD, mergeFunc: (VD, VD) => VD + ): VertexRDD[VD] = { + val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { + case Some(p) => vertices + case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size)) + } + val routingTables = createRoutingTables(edges, vPartitioned.partitioner.get) + val vertexPartitions = vPartitioned.zipPartitions(routingTables, preservesPartitioning = true) { + (vertexIter, routingTableIter) => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal)) } - val vertexPartitions = partitioned.mapPartitions( - iter => Iterator(VertexPartition(iter)), - preservesPartitioning = true) new VertexRDD(vertexPartitions) } /** - * Constructs a VertexRDD from the vertex IDs in `vids`, taking attributes from `rdd` and using - * `defaultVal` otherwise. + * Constructs a `VertexRDD` containing all vertices referred to in `edges`. The vertices will be + * created with the attribute `defaultVal`. The resulting `VertexRDD` will be joinable with + * `edges`. + * + * @tparam VD the vertex attribute type + * + * @param edges the [[EdgeRDD]] referring to the vertices to create + * @param numPartitions the desired number of partitions for the resulting `VertexRDD` + * @param defaultVal the vertex attribute to use when creating missing vertices */ - def apply[VD: ClassTag](vids: RDD[VertexId], rdd: RDD[(VertexId, VD)], defaultVal: VD) - : VertexRDD[VD] = { - VertexRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) => - value.getOrElse(default) - } + def fromEdges[VD: ClassTag]( + edges: EdgeRDD[_, _], numPartitions: Int, defaultVal: VD): VertexRDD[VD] = { + val routingTables = createRoutingTables(edges, new HashPartitioner(numPartitions)) + val vertexPartitions = routingTables.mapPartitions({ routingTableIter => + val routingTable = + if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty + Iterator(ShippableVertexPartition(Iterator.empty, routingTable, defaultVal)) + }, preservesPartitioning = true) + new VertexRDD(vertexPartitions) + } + + private def createRoutingTables( + edges: EdgeRDD[_, _], vertexPartitioner: Partitioner): RDD[RoutingTablePartition] = { + // Determine which vertices each edge partition needs by creating a mapping from vid to pid. + val vid2pid = edges.partitionsRDD.mapPartitions(_.flatMap( + Function.tupled(RoutingTablePartition.edgePartitionToMsgs))) + .setName("VertexRDD.createRoutingTables - vid2pid (aggregation)") + + val numEdgePartitions = edges.partitions.size + vid2pid.copartitionWithVertices(vertexPartitioner).mapPartitions( + iter => Iterator(RoutingTablePartition.fromMsgs(numEdgePartitions, iter)), + preservesPartitioning = true) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index b7c472e905a9b..871e81f8d245c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -17,39 +17,86 @@ package org.apache.spark.graphx.impl -import scala.reflect.ClassTag +import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** - * A collection of edges stored in 3 large columnar arrays (src, dst, attribute). The arrays are - * clustered by src. + * A collection of edges stored in columnar format, along with any vertex attributes referenced. The + * edges are stored in 3 large columnar arrays (src, dst, attribute). The arrays are clustered by + * src. There is an optional active vertex set for filtering computation on the edges. + * + * @tparam ED the edge attribute type + * @tparam VD the vertex attribute type * * @param srcIds the source vertex id of each edge * @param dstIds the destination vertex id of each edge * @param data the attribute associated with each edge * @param index a clustered index on source vertex id - * @tparam ED the edge attribute type. + * @param vertices a map from referenced vertex ids to their corresponding attributes. Must + * contain all vertex ids from `srcIds` and `dstIds`, though not necessarily valid attributes for + * those vertex ids. The mask is not used. + * @param activeSet an optional active vertex set for filtering computation on the edges */ private[graphx] -class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( +class EdgePartition[ + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( @transient val srcIds: Array[VertexId], @transient val dstIds: Array[VertexId], @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable { + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], + @transient val vertices: VertexPartition[VD], + @transient val activeSet: Option[VertexSet] = None + ) extends Serializable { + + /** Return a new `EdgePartition` with the specified edge data. */ + def withData[ED2: ClassTag](data_ : Array[ED2]): EdgePartition[ED2, VD] = { + new EdgePartition(srcIds, dstIds, data_, index, vertices, activeSet) + } + + /** Return a new `EdgePartition` with the specified vertex partition. */ + def withVertices[VD2: ClassTag]( + vertices_ : VertexPartition[VD2]): EdgePartition[ED, VD2] = { + new EdgePartition(srcIds, dstIds, data, index, vertices_, activeSet) + } + + /** Return a new `EdgePartition` with the specified active set, provided as an iterator. */ + def withActiveSet(iter: Iterator[VertexId]): EdgePartition[ED, VD] = { + val newActiveSet = new VertexSet + iter.foreach(newActiveSet.add(_)) + new EdgePartition(srcIds, dstIds, data, index, vertices, Some(newActiveSet)) + } + + /** Return a new `EdgePartition` with the specified active set. */ + def withActiveSet(activeSet_ : Option[VertexSet]): EdgePartition[ED, VD] = { + new EdgePartition(srcIds, dstIds, data, index, vertices, activeSet_) + } + + /** Return a new `EdgePartition` with updates to vertex attributes specified in `iter`. */ + def updateVertices(iter: Iterator[(VertexId, VD)]): EdgePartition[ED, VD] = { + this.withVertices(vertices.innerJoinKeepLeft(iter)) + } + + /** Look up vid in activeSet, throwing an exception if it is None. */ + def isActive(vid: VertexId): Boolean = { + activeSet.get.contains(vid) + } + + /** The number of active vertices, if any exist. */ + def numActives: Option[Int] = activeSet.map(_.size) /** * Reverse all the edges in this partition. * * @return a new edge partition with all edges reversed. */ - def reverse: EdgePartition[ED] = { - val builder = new EdgePartitionBuilder(size) + def reverse: EdgePartition[ED, VD] = { + val builder = new EdgePartitionBuilder(size)(classTag[ED], classTag[VD]) for (e <- iterator) { builder.add(e.dstId, e.srcId, e.attr) } - builder.toEdgePartition + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** @@ -64,7 +111,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @return a new edge partition with the result of the function `f` * applied to each edge */ - def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2] = { + def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2, VD] = { val newData = new Array[ED2](data.size) val edge = new Edge[ED]() val size = data.size @@ -76,7 +123,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) newData(i) = f(edge) i += 1 } - new EdgePartition(srcIds, dstIds, newData, index) + this.withData(newData) } /** @@ -91,7 +138,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @tparam ED2 the type of the new attribute * @return a new edge partition with the attribute values replaced */ - def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2] = { + def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2, VD] = { // Faster than iter.toArray, because the expected size is known. val newData = new Array[ED2](data.size) var i = 0 @@ -100,7 +147,23 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) i += 1 } assert(newData.size == i) - new EdgePartition(srcIds, dstIds, newData, index) + this.withData(newData) + } + + /** + * Construct a new edge partition containing only the edges matching `epred` and where both + * vertices match `vpred`. + */ + def filter( + epred: EdgeTriplet[VD, ED] => Boolean, + vpred: (VertexId, VD) => Boolean): EdgePartition[ED, VD] = { + val filtered = tripletIterator().filter(et => + vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et)) + val builder = new EdgePartitionBuilder[ED, VD] + for (e <- filtered) { + builder.add(e.srcId, e.dstId, e.attr) + } + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** @@ -119,8 +182,8 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * @param merge a commutative associative merge operation * @return a new edge partition without duplicate edges */ - def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = { - val builder = new EdgePartitionBuilder[ED] + def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED, VD] = { + val builder = new EdgePartitionBuilder[ED, VD] var currSrcId: VertexId = null.asInstanceOf[VertexId] var currDstId: VertexId = null.asInstanceOf[VertexId] var currAttr: ED = null.asInstanceOf[ED] @@ -141,11 +204,11 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) if (size > 0) { builder.add(currSrcId, currDstId, currAttr) } - builder.toEdgePartition + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** - * Apply `f` to all edges present in both `this` and `other` and return a new EdgePartition + * Apply `f` to all edges present in both `this` and `other` and return a new `EdgePartition` * containing the resulting edges. * * If there are multiple edges with the same src and dst in `this`, `f` will be invoked once for @@ -155,9 +218,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * once. */ def innerJoin[ED2: ClassTag, ED3: ClassTag] - (other: EdgePartition[ED2]) - (f: (VertexId, VertexId, ED, ED2) => ED3): EdgePartition[ED3] = { - val builder = new EdgePartitionBuilder[ED3] + (other: EdgePartition[ED2, _]) + (f: (VertexId, VertexId, ED, ED2) => ED3): EdgePartition[ED3, VD] = { + val builder = new EdgePartitionBuilder[ED3, VD] var i = 0 var j = 0 // For i = index of each edge in `this`... @@ -175,7 +238,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) } i += 1 } - builder.toEdgePartition + builder.toEdgePartition.withVertices(vertices).withActiveSet(activeSet) } /** @@ -183,7 +246,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * * @return size of the partition */ - def size: Int = srcIds.size + val size: Int = srcIds.size /** The number of unique source vertices in the partition. */ def indexSize: Int = index.size @@ -211,10 +274,35 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) } } + /** + * Get an iterator over the edge triplets in this partition. + * + * It is safe to keep references to the objects from this iterator. + */ + def tripletIterator( + includeSrc: Boolean = true, includeDst: Boolean = true): Iterator[EdgeTriplet[VD, ED]] = { + new EdgeTripletIterator(this, includeSrc, includeDst) + } + + /** + * Upgrade the given edge iterator into a triplet iterator. + * + * Be careful not to keep references to the objects from this iterator. To improve GC performance + * the same object is re-used in `next()`. + */ + def upgradeIterator( + edgeIter: Iterator[Edge[ED]], includeSrc: Boolean = true, includeDst: Boolean = true) + : Iterator[EdgeTriplet[VD, ED]] = { + new ReusingEdgeTripletIterator(edgeIter, this, includeSrc, includeDst) + } + /** * Get an iterator over the edges in this partition whose source vertex ids match srcIdPred. The * iterator is generated using an index scan, so it is efficient at skipping edges that don't * match srcIdPred. + * + * Be careful not to keep references to the objects from this iterator. To improve GC performance + * the same object is re-used in `next()`. */ def indexIterator(srcIdPred: VertexId => Boolean): Iterator[Edge[ED]] = index.iterator.filter(kv => srcIdPred(kv._1)).flatMap(Function.tupled(clusterIterator)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 63ccccb056b48..ecb49bef42e45 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -20,12 +20,14 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import scala.util.Sorting +import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} + import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.PrimitiveVector private[graphx] -class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: Int = 64) { +class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( + size: Int = 64) { var edges = new PrimitiveVector[Edge[ED]](size) /** Add a new edge to the partition. */ @@ -33,7 +35,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: I edges += Edge(src, dst, d) } - def toEdgePartition: EdgePartition[ED] = { + def toEdgePartition: EdgePartition[ED, VD] = { val edgeArray = edges.trim().array Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering) val srcIds = new Array[VertexId](edgeArray.size) @@ -57,6 +59,14 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: I i += 1 } } - new EdgePartition(srcIds, dstIds, data, index) + + // Create and populate a VertexPartition with vids from the edges, but no attributes + val vidsIter = srcIds.iterator ++ dstIds.iterator + val vertexIds = new OpenHashSet[VertexId] + vidsIter.foreach(vid => vertexIds.add(vid)) + val vertices = new VertexPartition( + vertexIds, new Array[VD](vertexIds.capacity), vertexIds.getBitSet) + + new EdgePartition(srcIds, dstIds, data, index, vertices) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index 220a89d73d711..ebb0b9418d65d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -23,32 +23,62 @@ import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** - * The Iterator type returned when constructing edge triplets. This class technically could be - * an anonymous class in GraphImpl.triplets, but we name it here explicitly so it is easier to - * debug / profile. + * The Iterator type returned when constructing edge triplets. This could be an anonymous class in + * EdgePartition.tripletIterator, but we name it here explicitly so it is easier to debug / profile. */ private[impl] class EdgeTripletIterator[VD: ClassTag, ED: ClassTag]( - val vidToIndex: VertexIdToIndexMap, - val vertexArray: Array[VD], - val edgePartition: EdgePartition[ED]) + val edgePartition: EdgePartition[ED, VD], + val includeSrc: Boolean, + val includeDst: Boolean) extends Iterator[EdgeTriplet[VD, ED]] { // Current position in the array. private var pos = 0 - private val vmap = new PrimitiveKeyOpenHashMap[VertexId, VD](vidToIndex, vertexArray) - override def hasNext: Boolean = pos < edgePartition.size override def next() = { val triplet = new EdgeTriplet[VD, ED] triplet.srcId = edgePartition.srcIds(pos) - triplet.srcAttr = vmap(triplet.srcId) + if (includeSrc) { + triplet.srcAttr = edgePartition.vertices(triplet.srcId) + } triplet.dstId = edgePartition.dstIds(pos) - triplet.dstAttr = vmap(triplet.dstId) + if (includeDst) { + triplet.dstAttr = edgePartition.vertices(triplet.dstId) + } triplet.attr = edgePartition.data(pos) pos += 1 triplet } } + +/** + * An Iterator type for internal use that reuses EdgeTriplet objects. This could be an anonymous + * class in EdgePartition.upgradeIterator, but we name it here explicitly so it is easier to debug / + * profile. + */ +private[impl] +class ReusingEdgeTripletIterator[VD: ClassTag, ED: ClassTag]( + val edgeIter: Iterator[Edge[ED]], + val edgePartition: EdgePartition[ED, VD], + val includeSrc: Boolean, + val includeDst: Boolean) + extends Iterator[EdgeTriplet[VD, ED]] { + + private val triplet = new EdgeTriplet[VD, ED] + + override def hasNext = edgeIter.hasNext + + override def next() = { + triplet.set(edgeIter.next()) + if (includeSrc) { + triplet.srcAttr = edgePartition.vertices(triplet.srcId) + } + if (includeDst) { + triplet.dstAttr = edgePartition.vertices(triplet.dstId) + } + triplet + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 9eabccdee48db..2f2d0e03fd7b5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -19,54 +19,45 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} -import org.apache.spark.util.collection.PrimitiveVector -import org.apache.spark.{HashPartitioner, Partitioner} +import org.apache.spark.HashPartitioner import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{RDD, ShuffledRDD} +import org.apache.spark.storage.StorageLevel + import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl._ import org.apache.spark.graphx.impl.MsgRDDFunctions._ import org.apache.spark.graphx.util.BytecodeUtils -import org.apache.spark.rdd.{ShuffledRDD, RDD} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.ClosureCleaner /** - * A graph that supports computation on graphs. + * An implementation of [[org.apache.spark.graphx.Graph]] to support computation on graphs. * - * Graphs are represented using two classes of data: vertex-partitioned and - * edge-partitioned. `vertices` contains vertex attributes, which are vertex-partitioned. `edges` - * contains edge attributes, which are edge-partitioned. For operations on vertex neighborhoods, - * vertex attributes are replicated to the edge partitions where they appear as sources or - * destinations. `routingTable` stores the routing information for shipping vertex attributes to - * edge partitions. `replicatedVertexView` stores a view of the replicated vertex attributes created - * using the routing table. + * Graphs are represented using two RDDs: `vertices`, which contains vertex attributes and the + * routing information for shipping vertex attributes to edge partitions, and + * `replicatedVertexView`, which contains edges and the vertex attributes mentioned by each edge. */ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( @transient val vertices: VertexRDD[VD], - @transient val edges: EdgeRDD[ED], - @transient val routingTable: RoutingTable, - @transient val replicatedVertexView: ReplicatedVertexView[VD]) + @transient val replicatedVertexView: ReplicatedVertexView[VD, ED]) extends Graph[VD, ED] with Serializable { /** Default constructor is provided to support serialization */ - protected def this() = this(null, null, null, null) + protected def this() = this(null, null) + + @transient override val edges: EdgeRDD[ED, VD] = replicatedVertexView.edges /** Return a RDD that brings edges together with their source and destination vertices. */ - @transient override val triplets: RDD[EdgeTriplet[VD, ED]] = { - val vdTag = classTag[VD] - val edTag = classTag[ED] - edges.partitionsRDD.zipPartitions( - replicatedVertexView.get(true, true), true) { (ePartIter, vPartIter) => - val (pid, ePart) = ePartIter.next() - val (_, vPart) = vPartIter.next() - new EdgeTripletIterator(vPart.index, vPart.values, ePart)(vdTag, edTag) - } + @transient override lazy val triplets: RDD[EdgeTriplet[VD, ED]] = { + replicatedVertexView.upgrade(vertices, true, true) + replicatedVertexView.edges.partitionsRDD.mapPartitions(_.flatMap { + case (pid, part) => part.tripletIterator() + }) } override def persist(newLevel: StorageLevel): Graph[VD, ED] = { vertices.persist(newLevel) - edges.persist(newLevel) + replicatedVertexView.edges.persist(newLevel) this } @@ -74,14 +65,15 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { vertices.unpersist(blocking) - replicatedVertexView.unpersist(blocking) + // TODO: unpersist the replicated vertices in `replicatedVertexView` but leave the edges alone this } override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = { - val numPartitions = edges.partitions.size + val numPartitions = replicatedVertexView.edges.partitions.size val edTag = classTag[ED] - val newEdges = new EdgeRDD(edges.map { e => + val vdTag = classTag[VD] + val newEdges = new EdgeRDD(replicatedVertexView.edges.map { e => val part: PartitionID = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) // Should we be using 3-tuple or an optimized class @@ -89,105 +81,79 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED]()(edTag) + val builder = new EdgePartitionBuilder[ED, VD]()(edTag, vdTag) iter.foreach { message => val data = message.data builder.add(data._1, data._2, data._3) } val edgePartition = builder.toEdgePartition Iterator((pid, edgePartition)) - }, preservesPartitioning = true).cache()) - GraphImpl(vertices, newEdges) + }, preservesPartitioning = true)) + GraphImpl.fromExistingRDDs(vertices, newEdges) } override def reverse: Graph[VD, ED] = { - val newETable = edges.mapEdgePartitions((pid, part) => part.reverse) - GraphImpl(vertices, newETable) + new GraphImpl(vertices.reverseRoutingTables(), replicatedVertexView.reverse()) } override def mapVertices[VD2: ClassTag](f: (VertexId, VD) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { + vertices.cache() // The map preserves type, so we can use incremental replication val newVerts = vertices.mapVertexPartitions(_.map(f)).cache() val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) - val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, routingTable, - Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) + val newReplicatedVertexView = replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2, ED]] + .updateVertices(changedVerts) + new GraphImpl(newVerts, newReplicatedVertexView) } else { // The map does not preserve type, so we must re-replicate all vertices - GraphImpl(vertices.mapVertexPartitions(_.map(f)), edges, routingTable) + GraphImpl(vertices.mapVertexPartitions(_.map(f)), replicatedVertexView.edges) } } override def mapEdges[ED2: ClassTag]( f: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2] = { - val newETable = edges.mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) - new GraphImpl(vertices, newETable , routingTable, replicatedVertexView) + val newEdges = replicatedVertexView.edges + .mapEdgePartitions((pid, part) => part.map(f(pid, part.iterator))) + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) } override def mapTriplets[ED2: ClassTag]( f: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2]): Graph[VD, ED2] = { - val newEdgePartitions = - edges.partitionsRDD.zipPartitions(replicatedVertexView.get(true, true), true) { - (ePartIter, vTableReplicatedIter) => - val (ePid, edgePartition) = ePartIter.next() - val (vPid, vPart) = vTableReplicatedIter.next() - assert(!vTableReplicatedIter.hasNext) - assert(ePid == vPid) - val et = new EdgeTriplet[VD, ED] - val inputIterator = edgePartition.iterator.map { e => - et.set(e) - et.srcAttr = vPart(e.srcId) - et.dstAttr = vPart(e.dstId) - et - } - // Apply the user function to the vertex partition - val outputIter = f(ePid, inputIterator) - // Consume the iterator to update the edge attributes - val newEdgePartition = edgePartition.map(outputIter) - Iterator((ePid, newEdgePartition)) - } - new GraphImpl(vertices, new EdgeRDD(newEdgePartitions), routingTable, replicatedVertexView) + vertices.cache() + val mapUsesSrcAttr = accessesVertexAttr(f, "srcAttr") + val mapUsesDstAttr = accessesVertexAttr(f, "dstAttr") + replicatedVertexView.upgrade(vertices, mapUsesSrcAttr, mapUsesDstAttr) + val newEdges = replicatedVertexView.edges.mapEdgePartitions { (pid, part) => + part.map(f(pid, part.tripletIterator(mapUsesSrcAttr, mapUsesDstAttr))) + } + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) } override def subgraph( epred: EdgeTriplet[VD, ED] => Boolean = x => true, vpred: (VertexId, VD) => Boolean = (a, b) => true): Graph[VD, ED] = { + vertices.cache() // Filter the vertices, reusing the partitioner and the index from this graph val newVerts = vertices.mapVertexPartitions(_.filter(vpred)) - - // Filter the edges - val edTag = classTag[ED] - val newEdges = new EdgeRDD[ED](triplets.filter { et => - vpred(et.srcId, et.srcAttr) && vpred(et.dstId, et.dstAttr) && epred(et) - }.mapPartitionsWithIndex( { (pid, iter) => - val builder = new EdgePartitionBuilder[ED]()(edTag) - iter.foreach { et => builder.add(et.srcId, et.dstId, et.attr) } - val edgePartition = builder.toEdgePartition - Iterator((pid, edgePartition)) - }, preservesPartitioning = true)).cache() - - // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been - // removed will be ignored, since we only refer to replicated vertices when they are adjacent to - // an edge. - new GraphImpl(newVerts, newEdges, new RoutingTable(newEdges, newVerts), replicatedVertexView) - } // end of subgraph + // Filter the triplets. We must always upgrade the triplet view fully because vpred always runs + // on both src and dst vertices + replicatedVertexView.upgrade(vertices, true, true) + val newEdges = replicatedVertexView.edges.filter(epred, vpred) + new GraphImpl(newVerts, replicatedVertexView.withEdges(newEdges)) + } override def mask[VD2: ClassTag, ED2: ClassTag] ( other: Graph[VD2, ED2]): Graph[VD, ED] = { val newVerts = vertices.innerJoin(other.vertices) { (vid, v, w) => v } - val newEdges = edges.innerJoin(other.edges) { (src, dst, v, w) => v } - // Reuse the previous ReplicatedVertexView unmodified. The replicated vertices that have been - // removed will be ignored, since we only refer to replicated vertices when they are adjacent to - // an edge. - new GraphImpl(newVerts, newEdges, routingTable, replicatedVertexView) + val newEdges = replicatedVertexView.edges.innerJoin(other.edges) { (src, dst, v, w) => v } + new GraphImpl(newVerts, replicatedVertexView.withEdges(newEdges)) } override def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED] = { - ClosureCleaner.clean(merge) - val newETable = edges.mapEdgePartitions((pid, part) => part.groupEdges(merge)) - new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) + val newEdges = replicatedVertexView.edges.mapEdgePartitions( + (pid, part) => part.groupEdges(merge)) + new GraphImpl(vertices, replicatedVertexView.withEdges(newEdges)) } // /////////////////////////////////////////////////////////////////////////////////////////////// @@ -199,68 +165,58 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( reduceFunc: (A, A) => A, activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = { - ClosureCleaner.clean(mapFunc) - ClosureCleaner.clean(reduceFunc) + vertices.cache() // For each vertex, replicate its attribute only to partitions where it is // in the relevant position in an edge. val mapUsesSrcAttr = accessesVertexAttr(mapFunc, "srcAttr") val mapUsesDstAttr = accessesVertexAttr(mapFunc, "dstAttr") - val vs = activeSetOpt match { + replicatedVertexView.upgrade(vertices, mapUsesSrcAttr, mapUsesDstAttr) + val view = activeSetOpt match { case Some((activeSet, _)) => - replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr, activeSet) + replicatedVertexView.withActiveSet(activeSet) case None => - replicatedVertexView.get(mapUsesSrcAttr, mapUsesDstAttr) + replicatedVertexView } val activeDirectionOpt = activeSetOpt.map(_._2) // Map and combine. - val preAgg = edges.partitionsRDD.zipPartitions(vs, true) { (ePartIter, vPartIter) => - val (ePid, edgePartition) = ePartIter.next() - val (vPid, vPart) = vPartIter.next() - assert(!vPartIter.hasNext) - assert(ePid == vPid) - // Choose scan method - val activeFraction = vPart.numActives.getOrElse(0) / edgePartition.indexSize.toFloat - val edgeIter = activeDirectionOpt match { - case Some(EdgeDirection.Both) => - if (activeFraction < 0.8) { - edgePartition.indexIterator(srcVertexId => vPart.isActive(srcVertexId)) - .filter(e => vPart.isActive(e.dstId)) - } else { - edgePartition.iterator.filter(e => vPart.isActive(e.srcId) && vPart.isActive(e.dstId)) - } - case Some(EdgeDirection.Either) => - // TODO: Because we only have a clustered index on the source vertex ID, we can't filter - // the index here. Instead we have to scan all edges and then do the filter. - edgePartition.iterator.filter(e => vPart.isActive(e.srcId) || vPart.isActive(e.dstId)) - case Some(EdgeDirection.Out) => - if (activeFraction < 0.8) { - edgePartition.indexIterator(srcVertexId => vPart.isActive(srcVertexId)) - } else { - edgePartition.iterator.filter(e => vPart.isActive(e.srcId)) - } - case Some(EdgeDirection.In) => - edgePartition.iterator.filter(e => vPart.isActive(e.dstId)) - case _ => // None - edgePartition.iterator - } - - // Scan edges and run the map function - val et = new EdgeTriplet[VD, ED] - val mapOutputs = edgeIter.flatMap { e => - et.set(e) - if (mapUsesSrcAttr) { - et.srcAttr = vPart(e.srcId) - } - if (mapUsesDstAttr) { - et.dstAttr = vPart(e.dstId) + val preAgg = view.edges.partitionsRDD.mapPartitions(_.flatMap { + case (pid, edgePartition) => + // Choose scan method + val activeFraction = edgePartition.numActives.getOrElse(0) / edgePartition.indexSize.toFloat + val edgeIter = activeDirectionOpt match { + case Some(EdgeDirection.Both) => + if (activeFraction < 0.8) { + edgePartition.indexIterator(srcVertexId => edgePartition.isActive(srcVertexId)) + .filter(e => edgePartition.isActive(e.dstId)) + } else { + edgePartition.iterator.filter(e => + edgePartition.isActive(e.srcId) && edgePartition.isActive(e.dstId)) + } + case Some(EdgeDirection.Either) => + // TODO: Because we only have a clustered index on the source vertex ID, we can't filter + // the index here. Instead we have to scan all edges and then do the filter. + edgePartition.iterator.filter(e => + edgePartition.isActive(e.srcId) || edgePartition.isActive(e.dstId)) + case Some(EdgeDirection.Out) => + if (activeFraction < 0.8) { + edgePartition.indexIterator(srcVertexId => edgePartition.isActive(srcVertexId)) + } else { + edgePartition.iterator.filter(e => edgePartition.isActive(e.srcId)) + } + case Some(EdgeDirection.In) => + edgePartition.iterator.filter(e => edgePartition.isActive(e.dstId)) + case _ => // None + edgePartition.iterator } - mapFunc(et) - } - // Note: This doesn't allow users to send messages to arbitrary vertices. - vPart.aggregateUsingIndex(mapOutputs, reduceFunc).iterator - } + + // Scan edges and run the map function + val mapOutputs = edgePartition.upgradeIterator(edgeIter, mapUsesSrcAttr, mapUsesDstAttr) + .flatMap(mapFunc(_)) + // Note: This doesn't allow users to send messages to arbitrary vertices. + edgePartition.vertices.aggregateUsingIndex(mapOutputs, reduceFunc).iterator + }).setName("GraphImpl.mapReduceTriplets - preAgg") // do the final reduction reusing the index map vertices.aggregateUsingIndex(preAgg, reduceFunc) @@ -268,20 +224,19 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def outerJoinVertices[U: ClassTag, VD2: ClassTag] (other: RDD[(VertexId, U)]) - (updateF: (VertexId, VD, Option[U]) => VD2): Graph[VD2, ED] = - { + (updateF: (VertexId, VD, Option[U]) => VD2): Graph[VD2, ED] = { if (classTag[VD] equals classTag[VD2]) { + vertices.cache() // updateF preserves type, so we can use incremental replication - val newVerts = vertices.leftJoin(other)(updateF) + val newVerts = vertices.leftJoin(other)(updateF).cache() val changedVerts = vertices.asInstanceOf[VertexRDD[VD2]].diff(newVerts) - val newReplicatedVertexView = new ReplicatedVertexView[VD2]( - changedVerts, edges, routingTable, - Some(replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2]])) - new GraphImpl(newVerts, edges, routingTable, newReplicatedVertexView) + val newReplicatedVertexView = replicatedVertexView.asInstanceOf[ReplicatedVertexView[VD2, ED]] + .updateVertices(changedVerts) + new GraphImpl(newVerts, newReplicatedVertexView) } else { // updateF does not preserve type, so we must re-replicate all vertices val newVerts = vertices.leftJoin(other)(updateF) - GraphImpl(newVerts, edges, routingTable) + GraphImpl(newVerts, replicatedVertexView.edges) } } @@ -298,73 +253,68 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( object GraphImpl { + /** Create a graph from edges, setting referenced vertices to `defaultVertexAttr`. */ def apply[VD: ClassTag, ED: ClassTag]( edges: RDD[Edge[ED]], - defaultVertexAttr: VD): GraphImpl[VD, ED] = - { + defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(createEdgeRDD(edges), defaultVertexAttr) } + /** Create a graph from EdgePartitions, setting referenced vertices to `defaultVertexAttr`. */ def fromEdgePartitions[VD: ClassTag, ED: ClassTag]( - edgePartitions: RDD[(PartitionID, EdgePartition[ED])], + edgePartitions: RDD[(PartitionID, EdgePartition[ED, VD])], defaultVertexAttr: VD): GraphImpl[VD, ED] = { fromEdgeRDD(new EdgeRDD(edgePartitions), defaultVertexAttr) } + /** Create a graph from vertices and edges, setting missing vertices to `defaultVertexAttr`. */ def apply[VD: ClassTag, ED: ClassTag]( vertices: RDD[(VertexId, VD)], edges: RDD[Edge[ED]], - defaultVertexAttr: VD): GraphImpl[VD, ED] = - { - val edgeRDD = createEdgeRDD(edges).cache() - - // Get the set of all vids - val partitioner = Partitioner.defaultPartitioner(vertices) - val vPartitioned = vertices.partitionBy(partitioner) - val vidsFromEdges = collectVertexIdsFromEdges(edgeRDD, partitioner) - val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) => - vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1) - } - - val vertexRDD = VertexRDD(vids, vPartitioned, defaultVertexAttr) - + defaultVertexAttr: VD): GraphImpl[VD, ED] = { + val edgeRDD = createEdgeRDD(edges)(classTag[ED], classTag[VD]).cache() + val vertexRDD = VertexRDD(vertices, edgeRDD, defaultVertexAttr) GraphImpl(vertexRDD, edgeRDD) } + /** Create a graph from a VertexRDD and an EdgeRDD with arbitrary replicated vertices. */ def apply[VD: ClassTag, ED: ClassTag]( vertices: VertexRDD[VD], - edges: EdgeRDD[ED]): GraphImpl[VD, ED] = { - // Cache RDDs that are referenced multiple times - edges.cache() - - GraphImpl(vertices, edges, new RoutingTable(edges, vertices)) + edges: EdgeRDD[ED, _]): GraphImpl[VD, ED] = { + // Convert the vertex partitions in edges to the correct type + val newEdges = edges.mapEdgePartitions( + (pid, part) => part.withVertices(part.vertices.map( + (vid, attr) => null.asInstanceOf[VD]))) + GraphImpl.fromExistingRDDs(vertices, newEdges) } - def apply[VD: ClassTag, ED: ClassTag]( + /** + * Create a graph from a VertexRDD and an EdgeRDD with the same replicated vertex type as the + * vertices. + */ + def fromExistingRDDs[VD: ClassTag, ED: ClassTag]( vertices: VertexRDD[VD], - edges: EdgeRDD[ED], - routingTable: RoutingTable): GraphImpl[VD, ED] = { - // Cache RDDs that are referenced multiple times. `routingTable` is cached by default, so we - // don't cache it explicitly. - vertices.cache() - edges.cache() - - new GraphImpl( - vertices, edges, routingTable, new ReplicatedVertexView(vertices, edges, routingTable)) + edges: EdgeRDD[ED, VD]): GraphImpl[VD, ED] = { + new GraphImpl(vertices, new ReplicatedVertexView(edges)) } /** - * Create the edge RDD, which is much more efficient for Java heap storage than the normal edges - * data structure (RDD[(VertexId, VertexId, ED)]). - * - * The edge RDD contains multiple partitions, and each partition contains only one RDD key-value - * pair: the key is the partition id, and the value is an EdgePartition object containing all the - * edges in a partition. + * Create a graph from an EdgeRDD with the correct vertex type, setting missing vertices to + * `defaultVertexAttr`. The vertices will have the same number of partitions as the EdgeRDD. */ - private def createEdgeRDD[ED: ClassTag]( - edges: RDD[Edge[ED]]): EdgeRDD[ED] = { + private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( + edges: EdgeRDD[ED, VD], + defaultVertexAttr: VD): GraphImpl[VD, ED] = { + edges.cache() + val vertices = VertexRDD.fromEdges(edges, edges.partitions.size, defaultVertexAttr) + fromExistingRDDs(vertices, edges) + } + + /** Create an EdgeRDD from a set of edges. */ + private def createEdgeRDD[ED: ClassTag, VD: ClassTag]( + edges: RDD[Edge[ED]]): EdgeRDD[ED, VD] = { val edgePartitions = edges.mapPartitionsWithIndex { (pid, iter) => - val builder = new EdgePartitionBuilder[ED] + val builder = new EdgePartitionBuilder[ED, VD] iter.foreach { e => builder.add(e.srcId, e.dstId, e.attr) } @@ -373,24 +323,4 @@ object GraphImpl { new EdgeRDD(edgePartitions) } - private def fromEdgeRDD[VD: ClassTag, ED: ClassTag]( - edges: EdgeRDD[ED], - defaultVertexAttr: VD): GraphImpl[VD, ED] = { - edges.cache() - // Get the set of all vids - val vids = collectVertexIdsFromEdges(edges, new HashPartitioner(edges.partitions.size)) - // Create the VertexRDD. - val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr)) - GraphImpl(vertices, edges) - } - - /** Collects all vids mentioned in edges and partitions them by partitioner. */ - private def collectVertexIdsFromEdges( - edges: EdgeRDD[_], - partitioner: Partitioner): RDD[(VertexId, Int)] = { - // TODO: Consider doing map side distinct before shuffle. - new ShuffledRDD[VertexId, Int, (VertexId, Int)]( - edges.collectVertexIds.map(vid => (vid, 0)), partitioner) - .setSerializer(new VertexIdMsgSerializer) - } } // end of object GraphImpl diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index c45ba3d2f8c24..1c6d7e59e9a27 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -89,7 +89,6 @@ class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { } - private[graphx] object MsgRDDFunctions { implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = { @@ -99,18 +98,28 @@ object MsgRDDFunctions { implicit def rdd2vertexMessageRDDFunctions[T: ClassTag](rdd: RDD[VertexBroadcastMsg[T]]) = { new VertexBroadcastMsgRDDFunctions(rdd) } +} - def partitionForAggregation[T: ClassTag](msgs: RDD[(VertexId, T)], partitioner: Partitioner) = { - val rdd = new ShuffledRDD[VertexId, T, (VertexId, T)](msgs, partitioner) +private[graphx] +class VertexRDDFunctions[VD: ClassTag](self: RDD[(VertexId, VD)]) { + def copartitionWithVertices(partitioner: Partitioner): RDD[(VertexId, VD)] = { + val rdd = new ShuffledRDD[VertexId, VD, (VertexId, VD)](self, partitioner) // Set a custom serializer if the data is of int or double type. - if (classTag[T] == ClassTag.Int) { + if (classTag[VD] == ClassTag.Int) { rdd.setSerializer(new IntAggMsgSerializer) - } else if (classTag[T] == ClassTag.Long) { + } else if (classTag[VD] == ClassTag.Long) { rdd.setSerializer(new LongAggMsgSerializer) - } else if (classTag[T] == ClassTag.Double) { + } else if (classTag[VD] == ClassTag.Double) { rdd.setSerializer(new DoubleAggMsgSerializer) } rdd } } + +private[graphx] +object VertexRDDFunctions { + implicit def rdd2VertexRDDFunctions[VD: ClassTag](rdd: RDD[(VertexId, VD)]) = { + new VertexRDDFunctions(rdd) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index a8154b63ce5fb..3a0bba1b93b41 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -21,192 +21,102 @@ import scala.reflect.{classTag, ClassTag} import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.collection.{PrimitiveVector, OpenHashSet} import org.apache.spark.graphx._ /** - * A view of the vertices after they are shipped to the join sites specified in - * `vertexPlacement`. The resulting view is co-partitioned with `edges`. If `prevViewOpt` is - * specified, `updatedVerts` are treated as incremental updates to the previous view. Otherwise, a - * fresh view is created. - * - * The view is always cached (i.e., once it is evaluated, it remains materialized). This avoids - * constructing it twice if the user calls graph.triplets followed by graph.mapReduceTriplets, for - * example. However, it means iterative algorithms must manually call `Graph.unpersist` on previous - * iterations' graphs for best GC performance. See the implementation of - * [[org.apache.spark.graphx.Pregel]] for an example. + * Manages shipping vertex attributes to the edge partitions of an + * [[org.apache.spark.graphx.EdgeRDD]]. Vertex attributes may be partially shipped to construct a + * triplet view with vertex attributes on only one side, and they may be updated. An active vertex + * set may additionally be shipped to the edge partitions. Be careful not to store a reference to + * `edges`, since it may be modified when the attribute shipping level is upgraded. */ private[impl] -class ReplicatedVertexView[VD: ClassTag]( - updatedVerts: VertexRDD[VD], - edges: EdgeRDD[_], - routingTable: RoutingTable, - prevViewOpt: Option[ReplicatedVertexView[VD]] = None) { +class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( + var edges: EdgeRDD[ED, VD], + var hasSrcId: Boolean = false, + var hasDstId: Boolean = false) { /** - * Within each edge partition, create a local map from vid to an index into the attribute - * array. Each map contains a superset of the vertices that it will receive, because it stores - * vids from both the source and destination of edges. It must always include both source and - * destination vids because some operations, such as GraphImpl.mapReduceTriplets, rely on this. + * Return a new `ReplicatedVertexView` with the specified `EdgeRDD`, which must have the same + * shipping level. */ - private val localVertexIdMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match { - case Some(prevView) => - prevView.localVertexIdMap - case None => - edges.partitionsRDD.mapPartitions(_.map { - case (pid, epart) => - val vidToIndex = new VertexIdToIndexMap - epart.foreach { e => - vidToIndex.add(e.srcId) - vidToIndex.add(e.dstId) - } - (pid, vidToIndex) - }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVertexIdMap") - } - - private lazy val bothAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(true, true) - private lazy val srcAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(true, false) - private lazy val dstAttrOnly: RDD[(PartitionID, VertexPartition[VD])] = create(false, true) - private lazy val noAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(false, false) - - def unpersist(blocking: Boolean = true): ReplicatedVertexView[VD] = { - bothAttrs.unpersist(blocking) - srcAttrOnly.unpersist(blocking) - dstAttrOnly.unpersist(blocking) - noAttrs.unpersist(blocking) - // Don't unpersist localVertexIdMap because a future ReplicatedVertexView may be using it - // without modification - this + def withEdges[VD2: ClassTag, ED2: ClassTag]( + edges_ : EdgeRDD[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { + new ReplicatedVertexView(edges_, hasSrcId, hasDstId) } - def get(includeSrc: Boolean, includeDst: Boolean): RDD[(PartitionID, VertexPartition[VD])] = { - (includeSrc, includeDst) match { - case (true, true) => bothAttrs - case (true, false) => srcAttrOnly - case (false, true) => dstAttrOnly - case (false, false) => noAttrs - } + /** + * Return a new `ReplicatedVertexView` where edges are reversed and shipping levels are swapped to + * match. + */ + def reverse() = { + val newEdges = edges.mapEdgePartitions((pid, part) => part.reverse) + new ReplicatedVertexView(newEdges, hasDstId, hasSrcId) } - def get( - includeSrc: Boolean, - includeDst: Boolean, - actives: VertexRDD[_]): RDD[(PartitionID, VertexPartition[VD])] = { - // Ship active sets to edge partitions using vertexPlacement, but ignoring includeSrc and - // includeDst. These flags govern attribute shipping, but the activeness of a vertex must be - // shipped to all edges mentioning that vertex, regardless of whether the vertex attribute is - // also shipped there. - val shippedActives = routingTable.get(true, true) - .zipPartitions(actives.partitionsRDD)(ReplicatedVertexView.buildActiveBuffer(_, _)) - .partitionBy(edges.partitioner.get) - // Update the view with shippedActives, setting activeness flags in the resulting - // VertexPartitions - get(includeSrc, includeDst).zipPartitions(shippedActives) { (viewIter, shippedActivesIter) => - val (pid, vPart) = viewIter.next() - val newPart = vPart.replaceActives(shippedActivesIter.flatMap(_._2.iterator)) - Iterator((pid, newPart)) + /** + * Upgrade the shipping level in-place to the specified levels by shipping vertex attributes from + * `vertices`. This operation modifies the `ReplicatedVertexView`, and callers can access `edges` + * afterwards to obtain the upgraded view. + */ + def upgrade(vertices: VertexRDD[VD], includeSrc: Boolean, includeDst: Boolean) { + val shipSrc = includeSrc && !hasSrcId + val shipDst = includeDst && !hasDstId + if (shipSrc || shipDst) { + val shippedVerts: RDD[(Int, VertexAttributeBlock[VD])] = + vertices.shipVertexAttributes(shipSrc, shipDst) + .setName("ReplicatedVertexView.upgrade(%s, %s) - shippedVerts %s %s (broadcast)".format( + includeSrc, includeDst, shipSrc, shipDst)) + .partitionBy(edges.partitioner.get) + val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { + (ePartIter, shippedVertsIter) => ePartIter.map { + case (pid, edgePartition) => + (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.iterator))) + } + }) + edges = newEdges + hasSrcId = includeSrc + hasDstId = includeDst } } - private def create(includeSrc: Boolean, includeDst: Boolean) - : RDD[(PartitionID, VertexPartition[VD])] = { - val vdTag = classTag[VD] - - // Ship vertex attributes to edge partitions according to vertexPlacement - val verts = updatedVerts.partitionsRDD - val shippedVerts = routingTable.get(includeSrc, includeDst) - .zipPartitions(verts)(ReplicatedVertexView.buildBuffer(_, _)(vdTag)) + /** + * Return a new `ReplicatedVertexView` where the `activeSet` in each edge partition contains only + * vertex ids present in `actives`. This ships a vertex id to all edge partitions where it is + * referenced, ignoring the attribute shipping level. + */ + def withActiveSet(actives: VertexRDD[_]): ReplicatedVertexView[VD, ED] = { + val shippedActives = actives.shipVertexIds() + .setName("ReplicatedVertexView.withActiveSet - shippedActives (broadcast)") .partitionBy(edges.partitioner.get) - // TODO: Consider using a specialized shuffler. - - prevViewOpt match { - case Some(prevView) => - // Update prevView with shippedVerts, setting staleness flags in the resulting - // VertexPartitions - prevView.get(includeSrc, includeDst).zipPartitions(shippedVerts) { - (prevViewIter, shippedVertsIter) => - val (pid, prevVPart) = prevViewIter.next() - val newVPart = prevVPart.innerJoinKeepLeft(shippedVertsIter.flatMap(_._2.iterator)) - Iterator((pid, newVPart)) - }.cache().setName("ReplicatedVertexView delta %s %s".format(includeSrc, includeDst)) - case None => - // Within each edge partition, place the shipped vertex attributes into the correct - // locations specified in localVertexIdMap - localVertexIdMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) => - val (pid, vidToIndex) = mapIter.next() - assert(!mapIter.hasNext) - // Populate the vertex array using the vidToIndex map - val vertexArray = vdTag.newArray(vidToIndex.capacity) - for ((_, block) <- shippedVertsIter) { - for (i <- 0 until block.vids.size) { - val vid = block.vids(i) - val attr = block.attrs(i) - val ind = vidToIndex.getPos(vid) - vertexArray(ind) = attr - } - } - val newVPart = new VertexPartition( - vidToIndex, vertexArray, vidToIndex.getBitSet)(vdTag) - Iterator((pid, newVPart)) - }.cache().setName("ReplicatedVertexView %s %s".format(includeSrc, includeDst)) - } - } -} - -private object ReplicatedVertexView { - protected def buildBuffer[VD: ClassTag]( - pid2vidIter: Iterator[Array[Array[VertexId]]], - vertexPartIter: Iterator[VertexPartition[VD]]) = { - val pid2vid: Array[Array[VertexId]] = pid2vidIter.next() - val vertexPart: VertexPartition[VD] = vertexPartIter.next() - - Iterator.tabulate(pid2vid.size) { pid => - val vidsCandidate = pid2vid(pid) - val size = vidsCandidate.length - val vids = new PrimitiveVector[VertexId](pid2vid(pid).size) - val attrs = new PrimitiveVector[VD](pid2vid(pid).size) - var i = 0 - while (i < size) { - val vid = vidsCandidate(i) - if (vertexPart.isDefined(vid)) { - vids += vid - attrs += vertexPart(vid) - } - i += 1 + val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedActives) { + (ePartIter, shippedActivesIter) => ePartIter.map { + case (pid, edgePartition) => + (pid, edgePartition.withActiveSet(shippedActivesIter.flatMap(_._2.iterator))) } - (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) - } + }) + new ReplicatedVertexView(newEdges, hasSrcId, hasDstId) } - protected def buildActiveBuffer( - pid2vidIter: Iterator[Array[Array[VertexId]]], - activePartIter: Iterator[VertexPartition[_]]) - : Iterator[(Int, Array[VertexId])] = { - val pid2vid: Array[Array[VertexId]] = pid2vidIter.next() - val activePart: VertexPartition[_] = activePartIter.next() + /** + * Return a new `ReplicatedVertexView` where vertex attributes in edge partition are updated using + * `updates`. This ships a vertex attribute only to the edge partitions where it is in the + * position(s) specified by the attribute shipping level. + */ + def updateVertices(updates: VertexRDD[VD]): ReplicatedVertexView[VD, ED] = { + val shippedVerts = updates.shipVertexAttributes(hasSrcId, hasDstId) + .setName("ReplicatedVertexView.updateVertices - shippedVerts %s %s (broadcast)".format( + hasSrcId, hasDstId)) + .partitionBy(edges.partitioner.get) - Iterator.tabulate(pid2vid.size) { pid => - val vidsCandidate = pid2vid(pid) - val size = vidsCandidate.length - val actives = new PrimitiveVector[VertexId](vidsCandidate.size) - var i = 0 - while (i < size) { - val vid = vidsCandidate(i) - if (activePart.isDefined(vid)) { - actives += vid - } - i += 1 + val newEdges = new EdgeRDD(edges.partitionsRDD.zipPartitions(shippedVerts) { + (ePartIter, shippedVertsIter) => ePartIter.map { + case (pid, edgePartition) => + (pid, edgePartition.updateVertices(shippedVertsIter.flatMap(_._2.iterator))) } - (pid, actives.trim().array) - } + }) + new ReplicatedVertexView(newEdges, hasSrcId, hasDstId) } } - -private[graphx] -class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], val attrs: Array[VD]) - extends Serializable { - def iterator: Iterator[(VertexId, VD)] = - (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala deleted file mode 100644 index 022d5668e2942..0000000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.graphx.impl - -import org.apache.spark.SparkContext._ -import org.apache.spark.graphx._ -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.collection.PrimitiveVector - -/** - * Stores the locations of edge-partition join sites for each vertex attribute; that is, the routing - * information for shipping vertex attributes to edge partitions. This is always cached because it - * may be used multiple times in ReplicatedVertexView -- once to ship the vertex attributes and - * (possibly) once to ship the active-set information. - */ -private[impl] -class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) { - - val bothAttrs: RDD[Array[Array[VertexId]]] = createPid2Vid(true, true) - val srcAttrOnly: RDD[Array[Array[VertexId]]] = createPid2Vid(true, false) - val dstAttrOnly: RDD[Array[Array[VertexId]]] = createPid2Vid(false, true) - val noAttrs: RDD[Array[Array[VertexId]]] = createPid2Vid(false, false) - - def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexId]]] = - (includeSrcAttr, includeDstAttr) match { - case (true, true) => bothAttrs - case (true, false) => srcAttrOnly - case (false, true) => dstAttrOnly - case (false, false) => noAttrs - } - - private def createPid2Vid( - includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexId]]] = { - // Determine which vertices each edge partition needs by creating a mapping from vid to pid. - val vid2pid: RDD[(VertexId, PartitionID)] = edges.partitionsRDD.mapPartitions { iter => - val (pid: PartitionID, edgePartition: EdgePartition[_]) = iter.next() - val numEdges = edgePartition.size - val vSet = new VertexSet - if (includeSrcAttr) { // Add src vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.srcIds(i)) - i += 1 - } - } - if (includeDstAttr) { // Add dst vertices to the set. - var i = 0 - while (i < numEdges) { - vSet.add(edgePartition.dstIds(i)) - i += 1 - } - } - vSet.iterator.map { vid => (vid, pid) } - } - - val numEdgePartitions = edges.partitions.size - vid2pid.partitionBy(vertices.partitioner.get).mapPartitions { iter => - val pid2vid = Array.fill(numEdgePartitions)(new PrimitiveVector[VertexId]) - for ((vid, pid) <- iter) { - pid2vid(pid) += vid - } - - Iterator(pid2vid.map(_.trim().array)) - }.cache().setName("RoutingTable %s %s".format(includeSrcAttr, includeDstAttr)) - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala new file mode 100644 index 0000000000000..927e32ad0f448 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.impl + +import scala.reflect.ClassTag + +import org.apache.spark.Partitioner +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.util.collection.{BitSet, PrimitiveVector} + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +/** + * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that + * the edge partition references `vid` in the specified `position` (src, dst, or both). +*/ +private[graphx] +class RoutingTableMessage( + var vid: VertexId, + var pid: PartitionID, + var position: Byte) + extends Product2[VertexId, (PartitionID, Byte)] with Serializable { + override def _1 = vid + override def _2 = (pid, position) + override def canEqual(that: Any): Boolean = that.isInstanceOf[RoutingTableMessage] +} + +private[graphx] +class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) { + /** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with the given `partitioner`. */ + def copartitionWithVertices(partitioner: Partitioner): RDD[RoutingTableMessage] = { + new ShuffledRDD[VertexId, (PartitionID, Byte), RoutingTableMessage](self, partitioner) + .setSerializer(new RoutingTableMessageSerializer) + } +} + +private[graphx] +object RoutingTableMessageRDDFunctions { + import scala.language.implicitConversions + + implicit def rdd2RoutingTableMessageRDDFunctions(rdd: RDD[RoutingTableMessage]) = { + new RoutingTableMessageRDDFunctions(rdd) + } +} + +private[graphx] +object RoutingTablePartition { + val empty: RoutingTablePartition = new RoutingTablePartition(Array.empty) + + /** Generate a `RoutingTableMessage` for each vertex referenced in `edgePartition`. */ + def edgePartitionToMsgs(pid: PartitionID, edgePartition: EdgePartition[_, _]) + : Iterator[RoutingTableMessage] = { + // Determine which positions each vertex id appears in using a map where the low 2 bits + // represent src and dst + val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + edgePartition.srcIds.iterator.foreach { srcId => + map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) + } + edgePartition.dstIds.iterator.foreach { dstId => + map.changeValue(dstId, 0x2, (b: Byte) => (b | 0x2).toByte) + } + map.iterator.map { vidAndPosition => + new RoutingTableMessage(vidAndPosition._1, pid, vidAndPosition._2) + } + } + + /** Build a `RoutingTablePartition` from `RoutingTableMessage`s. */ + def fromMsgs(numEdgePartitions: Int, iter: Iterator[RoutingTableMessage]) + : RoutingTablePartition = { + val pid2vid = Array.fill(numEdgePartitions)(new PrimitiveVector[VertexId]) + val srcFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) + val dstFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) + for (msg <- iter) { + pid2vid(msg.pid) += msg.vid + srcFlags(msg.pid) += (msg.position & 0x1) != 0 + dstFlags(msg.pid) += (msg.position & 0x2) != 0 + } + + new RoutingTablePartition(pid2vid.zipWithIndex.map { + case (vids, pid) => (vids.trim().array, toBitSet(srcFlags(pid)), toBitSet(dstFlags(pid))) + }) + } + + /** Compact the given vector of Booleans into a BitSet. */ + private def toBitSet(flags: PrimitiveVector[Boolean]): BitSet = { + val bitset = new BitSet(flags.size) + var i = 0 + while (i < flags.size) { + if (flags(i)) { + bitset.set(i) + } + i += 1 + } + bitset + } +} + +/** + * Stores the locations of edge-partition join sites for each vertex attribute in a particular + * vertex partition. This provides routing information for shipping vertex attributes to edge + * partitions. + */ +private[graphx] +class RoutingTablePartition( + private val routingTable: Array[(Array[VertexId], BitSet, BitSet)]) { + /** The maximum number of edge partitions this `RoutingTablePartition` is built to join with. */ + val numEdgePartitions: Int = routingTable.size + + /** Returns the number of vertices that will be sent to the specified edge partition. */ + def partitionSize(pid: PartitionID): Int = routingTable(pid)._1.size + + /** Returns an iterator over all vertex ids stored in this `RoutingTablePartition`. */ + def iterator: Iterator[VertexId] = routingTable.iterator.flatMap(_._1.iterator) + + /** Returns a new RoutingTablePartition reflecting a reversal of all edge directions. */ + def reverse: RoutingTablePartition = { + new RoutingTablePartition(routingTable.map { + case (vids, srcVids, dstVids) => (vids, dstVids, srcVids) + }) + } + + /** + * Runs `f` on each vertex id to be sent to the specified edge partition. Vertex ids can be + * filtered by the position they have in the edge partition. + */ + def foreachWithinEdgePartition + (pid: PartitionID, includeSrc: Boolean, includeDst: Boolean) + (f: VertexId => Unit) { + val (vidsCandidate, srcVids, dstVids) = routingTable(pid) + val size = vidsCandidate.length + if (includeSrc && includeDst) { + // Avoid checks for performance + vidsCandidate.iterator.foreach(f) + } else if (!includeSrc && !includeDst) { + // Do nothing + } else { + val relevantVids = if (includeSrc) srcVids else dstVids + relevantVids.iterator.foreach { i => f(vidsCandidate(i)) } + } + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 1de42eeca1f00..033237f597216 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -27,6 +27,35 @@ import scala.reflect.ClassTag import org.apache.spark.graphx._ import org.apache.spark.serializer._ +private[graphx] +class RoutingTableMessageSerializer extends Serializer with Serializable { + override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { + + override def serializeStream(s: OutputStream): SerializationStream = + new ShuffleSerializationStream(s) { + def writeObject[T: ClassTag](t: T): SerializationStream = { + val msg = t.asInstanceOf[RoutingTableMessage] + writeVarLong(msg.vid, optimizePositive = false) + writeUnsignedVarInt(msg.pid) + // TODO: Write only the bottom two bits of msg.position + s.write(msg.position) + this + } + } + + override def deserializeStream(s: InputStream): DeserializationStream = + new ShuffleDeserializationStream(s) { + override def readObject[T: ClassTag](): T = { + val a = readVarLong(optimizePositive = false) + val b = readUnsignedVarInt() + val c = s.read() + if (c == -1) throw new EOFException + new RoutingTableMessage(a, b, c.toByte).asInstanceOf[T] + } + } + } +} + private[graphx] class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala new file mode 100644 index 0000000000000..f4e221d4e05ae --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.impl + +import scala.reflect.ClassTag + +import org.apache.spark.util.collection.{BitSet, PrimitiveVector} + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +/** Stores vertex attributes to ship to an edge partition. */ +private[graphx] +class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], val attrs: Array[VD]) + extends Serializable { + def iterator: Iterator[(VertexId, VD)] = + (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) } +} + +private[graphx] +object ShippableVertexPartition { + /** Construct a `ShippableVertexPartition` from the given vertices without any routing table. */ + def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): ShippableVertexPartition[VD] = + apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD]) + + /** + * Construct a `ShippableVertexPartition` from the given vertices with the specified routing + * table, filling in missing vertices mentioned in the routing table using `defaultVal`. + */ + def apply[VD: ClassTag]( + iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD) + : ShippableVertexPartition[VD] = { + val fullIter = iter ++ routingTable.iterator.map(vid => (vid, defaultVal)) + val (index, values, mask) = VertexPartitionBase.initFrom(fullIter, (a: VD, b: VD) => a) + new ShippableVertexPartition(index, values, mask, routingTable) + } + + import scala.language.implicitConversions + + /** + * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a + * `ShippableVertexPartition`. + */ + implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) = + new ShippableVertexPartitionOps(partition) + + /** + * Implicit evidence that `ShippableVertexPartition` is a member of the + * `VertexPartitionBaseOpsConstructor` typeclass. This enables invoking `VertexPartitionBase` + * operations on a `ShippableVertexPartition` via an evidence parameter, as in + * [[VertexPartitionBaseOps]]. + */ + implicit object ShippableVertexPartitionOpsConstructor + extends VertexPartitionBaseOpsConstructor[ShippableVertexPartition] { + def toOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) + : VertexPartitionBaseOps[VD, ShippableVertexPartition] = shippablePartitionToOps(partition) + } +} + +/** + * A map from vertex id to vertex attribute that additionally stores edge partition join sites for + * each vertex attribute, enabling joining with an [[org.apache.spark.graphx.EdgeRDD]]. + */ +private[graphx] +class ShippableVertexPartition[VD: ClassTag]( + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet, + val routingTable: RoutingTablePartition) + extends VertexPartitionBase[VD] { + + /** Return a new ShippableVertexPartition with the specified routing table. */ + def withRoutingTable(routingTable_ : RoutingTablePartition): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, values, mask, routingTable_) + } + + /** + * Generate a `VertexAttributeBlock` for each edge partition keyed on the edge partition ID. The + * `VertexAttributeBlock` contains the vertex attributes from the current partition that are + * referenced in the specified positions in the edge partition. + */ + def shipVertexAttributes( + shipSrc: Boolean, shipDst: Boolean): Iterator[(PartitionID, VertexAttributeBlock[VD])] = { + Iterator.tabulate(routingTable.numEdgePartitions) { pid => + val initialSize = if (shipSrc && shipDst) routingTable.partitionSize(pid) else 64 + val vids = new PrimitiveVector[VertexId](initialSize) + val attrs = new PrimitiveVector[VD](initialSize) + var i = 0 + routingTable.foreachWithinEdgePartition(pid, shipSrc, shipDst) { vid => + if (isDefined(vid)) { + vids += vid + attrs += this(vid) + } + i += 1 + } + (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) + } + } + + /** + * Generate a `VertexId` array for each edge partition keyed on the edge partition ID. The array + * contains the visible vertex ids from the current partition that are referenced in the edge + * partition. + */ + def shipVertexIds(): Iterator[(PartitionID, Array[VertexId])] = { + Iterator.tabulate(routingTable.numEdgePartitions) { pid => + val vids = new PrimitiveVector[VertexId](routingTable.partitionSize(pid)) + var i = 0 + routingTable.foreachWithinEdgePartition(pid, true, true) { vid => + if (isDefined(vid)) { + vids += vid + } + i += 1 + } + (pid, vids.trim().array) + } + } +} + +private[graphx] class ShippableVertexPartitionOps[VD: ClassTag](self: ShippableVertexPartition[VD]) + extends VertexPartitionBaseOps[VD, ShippableVertexPartition](self) { + + def withIndex(index: VertexIdToIndexMap): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, self.values, self.mask, self.routingTable) + } + + def withValues[VD2: ClassTag](values: Array[VD2]): ShippableVertexPartition[VD2] = { + new ShippableVertexPartition(self.index, values, self.mask, self.routingTable) + } + + def withMask(mask: BitSet): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(self.index, self.values, mask, self.routingTable) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 7a54b413dc8ca..f1d174720a1ba 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -19,260 +19,59 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import org.apache.spark.Logging +import org.apache.spark.util.collection.BitSet + import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.BitSet private[graphx] object VertexPartition { - - def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): VertexPartition[VD] = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] - iter.foreach { case (k, v) => - map(k) = v - } - new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) - } - - def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) - : VertexPartition[VD] = - { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] - iter.foreach { case (k, v) => - map.setMerge(k, v, mergeFunc) - } - new VertexPartition(map.keySet, map._values, map.keySet.getBitSet) - } -} - - -private[graphx] -class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag]( - val index: VertexIdToIndexMap, - val values: Array[VD], - val mask: BitSet, - /** A set of vids of active vertices. May contain vids not in index due to join rewrite. */ - private val activeSet: Option[VertexSet] = None) - extends Logging { - - val capacity: Int = index.capacity - - def size: Int = mask.cardinality() - - /** Return the vertex attribute for the given vertex ID. */ - def apply(vid: VertexId): VD = values(index.getPos(vid)) - - def isDefined(vid: VertexId): Boolean = { - val pos = index.getPos(vid) - pos >= 0 && mask.get(pos) - } - - /** Look up vid in activeSet, throwing an exception if it is None. */ - def isActive(vid: VertexId): Boolean = { - activeSet.get.contains(vid) + /** Construct a `VertexPartition` from the given vertices. */ + def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]) + : VertexPartition[VD] = { + val (index, values, mask) = VertexPartitionBase.initFrom(iter) + new VertexPartition(index, values, mask) } - /** The number of active vertices, if any exist. */ - def numActives: Option[Int] = activeSet.map(_.size) + import scala.language.implicitConversions /** - * Pass each vertex attribute along with the vertex id through a map - * function and retain the original RDD's partitioning and index. - * - * @tparam VD2 the type returned by the map function - * - * @param f the function applied to each vertex id and vertex - * attribute in the RDD - * - * @return a new VertexPartition with values obtained by applying `f` to - * each of the entries in the original VertexRDD. The resulting - * VertexPartition retains the same index. + * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a + * `VertexPartition`. */ - def map[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexPartition[VD2] = { - // Construct a view of the map transformation - val newValues = new Array[VD2](capacity) - var i = mask.nextSetBit(0) - while (i >= 0) { - newValues(i) = f(index.getValue(i), values(i)) - i = mask.nextSetBit(i + 1) - } - new VertexPartition[VD2](index, newValues, mask) - } - - /** - * Restrict the vertex set to the set of vertices satisfying the given predicate. - * - * @param pred the user defined predicate - * - * @note The vertex set preserves the original index structure which means that the returned - * RDD can be easily joined with the original vertex-set. Furthermore, the filter only - * modifies the bitmap index and so no new values are allocated. - */ - def filter(pred: (VertexId, VD) => Boolean): VertexPartition[VD] = { - // Allocate the array to store the results into - val newMask = new BitSet(capacity) - // Iterate over the active bits in the old mask and evaluate the predicate - var i = mask.nextSetBit(0) - while (i >= 0) { - if (pred(index.getValue(i), values(i))) { - newMask.set(i) - } - i = mask.nextSetBit(i + 1) - } - new VertexPartition(index, values, newMask) - } + implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) = + new VertexPartitionOps(partition) /** - * Hides vertices that are the same between this and other. For vertices that are different, keeps - * the values from `other`. The indices of `this` and `other` must be the same. + * Implicit evidence that `VertexPartition` is a member of the `VertexPartitionBaseOpsConstructor` + * typeclass. This enables invoking `VertexPartitionBase` operations on a `VertexPartition` via an + * evidence parameter, as in [[VertexPartitionBaseOps]]. */ - def diff(other: VertexPartition[VD]): VertexPartition[VD] = { - if (index != other.index) { - logWarning("Diffing two VertexPartitions with different indexes is slow.") - diff(createUsingIndex(other.iterator)) - } else { - val newMask = mask & other.mask - var i = newMask.nextSetBit(0) - while (i >= 0) { - if (values(i) == other.values(i)) { - newMask.unset(i) - } - i = newMask.nextSetBit(i + 1) - } - new VertexPartition(index, other.values, newMask) - } - } - - /** Left outer join another VertexPartition. */ - def leftJoin[VD2: ClassTag, VD3: ClassTag] - (other: VertexPartition[VD2]) - (f: (VertexId, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { - if (index != other.index) { - logWarning("Joining two VertexPartitions with different indexes is slow.") - leftJoin(createUsingIndex(other.iterator))(f) - } else { - val newValues = new Array[VD3](capacity) - - var i = mask.nextSetBit(0) - while (i >= 0) { - val otherV: Option[VD2] = if (other.mask.get(i)) Some(other.values(i)) else None - newValues(i) = f(index.getValue(i), values(i), otherV) - i = mask.nextSetBit(i + 1) - } - new VertexPartition(index, newValues, mask) - } - } - - /** Left outer join another iterator of messages. */ - def leftJoin[VD2: ClassTag, VD3: ClassTag] - (other: Iterator[(VertexId, VD2)]) - (f: (VertexId, VD, Option[VD2]) => VD3): VertexPartition[VD3] = { - leftJoin(createUsingIndex(other))(f) - } - - /** Inner join another VertexPartition. */ - def innerJoin[U: ClassTag, VD2: ClassTag](other: VertexPartition[U]) - (f: (VertexId, VD, U) => VD2): VertexPartition[VD2] = { - if (index != other.index) { - logWarning("Joining two VertexPartitions with different indexes is slow.") - innerJoin(createUsingIndex(other.iterator))(f) - } else { - val newMask = mask & other.mask - val newValues = new Array[VD2](capacity) - var i = newMask.nextSetBit(0) - while (i >= 0) { - newValues(i) = f(index.getValue(i), values(i), other.values(i)) - i = newMask.nextSetBit(i + 1) - } - new VertexPartition(index, newValues, newMask) - } - } - - /** - * Inner join an iterator of messages. - */ - def innerJoin[U: ClassTag, VD2: ClassTag] - (iter: Iterator[Product2[VertexId, U]]) - (f: (VertexId, VD, U) => VD2): VertexPartition[VD2] = { - innerJoin(createUsingIndex(iter))(f) + implicit object VertexPartitionOpsConstructor + extends VertexPartitionBaseOpsConstructor[VertexPartition] { + def toOps[VD: ClassTag](partition: VertexPartition[VD]) + : VertexPartitionBaseOps[VD, VertexPartition] = partitionToOps(partition) } +} - /** - * Similar effect as aggregateUsingIndex((a, b) => a) - */ - def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexId, VD2]]) - : VertexPartition[VD2] = { - val newMask = new BitSet(capacity) - val newValues = new Array[VD2](capacity) - iter.foreach { case (vid, vdata) => - val pos = index.getPos(vid) - if (pos >= 0) { - newMask.set(pos) - newValues(pos) = vdata - } - } - new VertexPartition[VD2](index, newValues, newMask) - } +/** A map from vertex id to vertex attribute. */ +private[graphx] class VertexPartition[VD: ClassTag]( + val index: VertexIdToIndexMap, + val values: Array[VD], + val mask: BitSet) + extends VertexPartitionBase[VD] - /** - * Similar to innerJoin, but vertices from the left side that don't appear in iter will remain in - * the partition, hidden by the bitmask. - */ - def innerJoinKeepLeft(iter: Iterator[Product2[VertexId, VD]]): VertexPartition[VD] = { - val newMask = new BitSet(capacity) - val newValues = new Array[VD](capacity) - System.arraycopy(values, 0, newValues, 0, newValues.length) - iter.foreach { case (vid, vdata) => - val pos = index.getPos(vid) - if (pos >= 0) { - newMask.set(pos) - newValues(pos) = vdata - } - } - new VertexPartition(index, newValues, newMask) - } +private[graphx] class VertexPartitionOps[VD: ClassTag](self: VertexPartition[VD]) + extends VertexPartitionBaseOps[VD, VertexPartition](self) { - def aggregateUsingIndex[VD2: ClassTag]( - iter: Iterator[Product2[VertexId, VD2]], - reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = { - val newMask = new BitSet(capacity) - val newValues = new Array[VD2](capacity) - iter.foreach { product => - val vid = product._1 - val vdata = product._2 - val pos = index.getPos(vid) - if (pos >= 0) { - if (newMask.get(pos)) { - newValues(pos) = reduceFunc(newValues(pos), vdata) - } else { // otherwise just store the new value - newMask.set(pos) - newValues(pos) = vdata - } - } - } - new VertexPartition[VD2](index, newValues, newMask) + def withIndex(index: VertexIdToIndexMap): VertexPartition[VD] = { + new VertexPartition(index, self.values, self.mask) } - def replaceActives(iter: Iterator[VertexId]): VertexPartition[VD] = { - val newActiveSet = new VertexSet - iter.foreach(newActiveSet.add(_)) - new VertexPartition(index, values, mask, Some(newActiveSet)) + def withValues[VD2: ClassTag](values: Array[VD2]): VertexPartition[VD2] = { + new VertexPartition(self.index, values, self.mask) } - /** - * Construct a new VertexPartition whose index contains only the vertices in the mask. - */ - def reindex(): VertexPartition[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] - val arbitraryMerge = (a: VD, b: VD) => a - for ((k, v) <- this.iterator) { - hashMap.setMerge(k, v, arbitraryMerge) - } - new VertexPartition(hashMap.keySet, hashMap._values, hashMap.keySet.getBitSet) + def withMask(mask: BitSet): VertexPartition[VD] = { + new VertexPartition(self.index, self.values, mask) } - - def iterator: Iterator[(VertexId, VD)] = - mask.iterator.map(ind => (index.getValue(ind), values(ind))) - - def vidIterator: Iterator[VertexId] = mask.iterator.map(ind => index.getValue(ind)) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala new file mode 100644 index 0000000000000..8d9e0204d27f2 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.impl + +import scala.language.higherKinds +import scala.reflect.ClassTag + +import org.apache.spark.util.collection.BitSet + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +private[graphx] object VertexPartitionBase { + /** + * Construct the constituents of a VertexPartitionBase from the given vertices, merging duplicate + * entries arbitrarily. + */ + def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) + : (VertexIdToIndexMap, Array[VD], BitSet) = { + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + iter.foreach { pair => + map(pair._1) = pair._2 + } + (map.keySet, map._values, map.keySet.getBitSet) + } + + /** + * Construct the constituents of a VertexPartitionBase from the given vertices, merging duplicate + * entries using `mergeFunc`. + */ + def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) + : (VertexIdToIndexMap, Array[VD], BitSet) = { + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + iter.foreach { pair => + map.setMerge(pair._1, pair._2, mergeFunc) + } + (map.keySet, map._values, map.keySet.getBitSet) + } +} + +/** + * An abstract map from vertex id to vertex attribute. [[VertexPartition]] is the corresponding + * concrete implementation. [[VertexPartitionBaseOps]] provides a variety of operations for + * VertexPartitionBase and subclasses that provide implicit evidence of membership in the + * `VertexPartitionBaseOpsConstructor` typeclass (for example, + * [[VertexPartition.VertexPartitionOpsConstructor]]). + */ +private[graphx] abstract class VertexPartitionBase[@specialized(Long, Int, Double) VD: ClassTag] { + + def index: VertexIdToIndexMap + def values: Array[VD] + def mask: BitSet + + val capacity: Int = index.capacity + + def size: Int = mask.cardinality() + + /** Return the vertex attribute for the given vertex ID. */ + def apply(vid: VertexId): VD = values(index.getPos(vid)) + + def isDefined(vid: VertexId): Boolean = { + val pos = index.getPos(vid) + pos >= 0 && mask.get(pos) + } + + def iterator: Iterator[(VertexId, VD)] = + mask.iterator.map(ind => (index.getValue(ind), values(ind))) +} + +/** + * A typeclass for subclasses of `VertexPartitionBase` representing the ability to wrap them in a + * `VertexPartitionBaseOps`. + */ +private[graphx] trait VertexPartitionBaseOpsConstructor[T[X] <: VertexPartitionBase[X]] { + def toOps[VD: ClassTag](partition: T[VD]): VertexPartitionBaseOps[VD, T] +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala new file mode 100644 index 0000000000000..21ff615feca6c --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx.impl + +import scala.language.higherKinds +import scala.language.implicitConversions +import scala.reflect.ClassTag + +import org.apache.spark.Logging +import org.apache.spark.util.collection.BitSet + +import org.apache.spark.graphx._ +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap + +/** + * An class containing additional operations for subclasses of VertexPartitionBase that provide + * implicit evidence of membership in the `VertexPartitionBaseOpsConstructor` typeclass (for + * example, [[VertexPartition.VertexPartitionOpsConstructor]]). + */ +private[graphx] abstract class VertexPartitionBaseOps + [VD: ClassTag, Self[X] <: VertexPartitionBase[X] : VertexPartitionBaseOpsConstructor] + (self: Self[VD]) + extends Logging { + + def withIndex(index: VertexIdToIndexMap): Self[VD] + def withValues[VD2: ClassTag](values: Array[VD2]): Self[VD2] + def withMask(mask: BitSet): Self[VD] + + /** + * Pass each vertex attribute along with the vertex id through a map + * function and retain the original RDD's partitioning and index. + * + * @tparam VD2 the type returned by the map function + * + * @param f the function applied to each vertex id and vertex + * attribute in the RDD + * + * @return a new VertexPartition with values obtained by applying `f` to + * each of the entries in the original VertexRDD. The resulting + * VertexPartition retains the same index. + */ + def map[VD2: ClassTag](f: (VertexId, VD) => VD2): Self[VD2] = { + // Construct a view of the map transformation + val newValues = new Array[VD2](self.capacity) + var i = self.mask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(self.index.getValue(i), self.values(i)) + i = self.mask.nextSetBit(i + 1) + } + this.withValues(newValues) + } + + /** + * Restrict the vertex set to the set of vertices satisfying the given predicate. + * + * @param pred the user defined predicate + * + * @note The vertex set preserves the original index structure which means that the returned + * RDD can be easily joined with the original vertex-set. Furthermore, the filter only + * modifies the bitmap index and so no new values are allocated. + */ + def filter(pred: (VertexId, VD) => Boolean): Self[VD] = { + // Allocate the array to store the results into + val newMask = new BitSet(self.capacity) + // Iterate over the active bits in the old mask and evaluate the predicate + var i = self.mask.nextSetBit(0) + while (i >= 0) { + if (pred(self.index.getValue(i), self.values(i))) { + newMask.set(i) + } + i = self.mask.nextSetBit(i + 1) + } + this.withMask(newMask) + } + + /** + * Hides vertices that are the same between this and other. For vertices that are different, keeps + * the values from `other`. The indices of `this` and `other` must be the same. + */ + def diff(other: Self[VD]): Self[VD] = { + if (self.index != other.index) { + logWarning("Diffing two VertexPartitions with different indexes is slow.") + diff(createUsingIndex(other.iterator)) + } else { + val newMask = self.mask & other.mask + var i = newMask.nextSetBit(0) + while (i >= 0) { + if (self.values(i) == other.values(i)) { + newMask.unset(i) + } + i = newMask.nextSetBit(i + 1) + } + this.withValues(other.values).withMask(newMask) + } + } + + /** Left outer join another VertexPartition. */ + def leftJoin[VD2: ClassTag, VD3: ClassTag] + (other: Self[VD2]) + (f: (VertexId, VD, Option[VD2]) => VD3): Self[VD3] = { + if (self.index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + leftJoin(createUsingIndex(other.iterator))(f) + } else { + val newValues = new Array[VD3](self.capacity) + + var i = self.mask.nextSetBit(0) + while (i >= 0) { + val otherV: Option[VD2] = if (other.mask.get(i)) Some(other.values(i)) else None + newValues(i) = f(self.index.getValue(i), self.values(i), otherV) + i = self.mask.nextSetBit(i + 1) + } + this.withValues(newValues) + } + } + + /** Left outer join another iterator of messages. */ + def leftJoin[VD2: ClassTag, VD3: ClassTag] + (other: Iterator[(VertexId, VD2)]) + (f: (VertexId, VD, Option[VD2]) => VD3): Self[VD3] = { + leftJoin(createUsingIndex(other))(f) + } + + /** Inner join another VertexPartition. */ + def innerJoin[U: ClassTag, VD2: ClassTag] + (other: Self[U]) + (f: (VertexId, VD, U) => VD2): Self[VD2] = { + if (self.index != other.index) { + logWarning("Joining two VertexPartitions with different indexes is slow.") + innerJoin(createUsingIndex(other.iterator))(f) + } else { + val newMask = self.mask & other.mask + val newValues = new Array[VD2](self.capacity) + var i = newMask.nextSetBit(0) + while (i >= 0) { + newValues(i) = f(self.index.getValue(i), self.values(i), other.values(i)) + i = newMask.nextSetBit(i + 1) + } + this.withValues(newValues).withMask(newMask) + } + } + + /** + * Inner join an iterator of messages. + */ + def innerJoin[U: ClassTag, VD2: ClassTag] + (iter: Iterator[Product2[VertexId, U]]) + (f: (VertexId, VD, U) => VD2): Self[VD2] = { + innerJoin(createUsingIndex(iter))(f) + } + + /** + * Similar effect as aggregateUsingIndex((a, b) => a) + */ + def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexId, VD2]]) + : Self[VD2] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD2](self.capacity) + iter.foreach { pair => + val pos = self.index.getPos(pair._1) + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = pair._2 + } + } + this.withValues(newValues).withMask(newMask) + } + + /** + * Similar to innerJoin, but vertices from the left side that don't appear in iter will remain in + * the partition, hidden by the bitmask. + */ + def innerJoinKeepLeft(iter: Iterator[Product2[VertexId, VD]]): Self[VD] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD](self.capacity) + System.arraycopy(self.values, 0, newValues, 0, newValues.length) + iter.foreach { pair => + val pos = self.index.getPos(pair._1) + if (pos >= 0) { + newMask.set(pos) + newValues(pos) = pair._2 + } + } + this.withValues(newValues).withMask(newMask) + } + + def aggregateUsingIndex[VD2: ClassTag]( + iter: Iterator[Product2[VertexId, VD2]], + reduceFunc: (VD2, VD2) => VD2): Self[VD2] = { + val newMask = new BitSet(self.capacity) + val newValues = new Array[VD2](self.capacity) + iter.foreach { product => + val vid = product._1 + val vdata = product._2 + val pos = self.index.getPos(vid) + if (pos >= 0) { + if (newMask.get(pos)) { + newValues(pos) = reduceFunc(newValues(pos), vdata) + } else { // otherwise just store the new value + newMask.set(pos) + newValues(pos) = vdata + } + } + } + this.withValues(newValues).withMask(newMask) + } + + /** + * Construct a new VertexPartition whose index contains only the vertices in the mask. + */ + def reindex(): Self[VD] = { + val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val arbitraryMerge = (a: VD, b: VD) => a + for ((k, v) <- self.iterator) { + hashMap.setMerge(k, v, arbitraryMerge) + } + this.withIndex(hashMap.keySet).withValues(hashMap._values).withMask(hashMap.keySet.getBitSet) + } + + /** + * Converts a vertex partition (in particular, one of type `Self`) into a + * `VertexPartitionBaseOps`. Within this class, this allows chaining the methods defined above, + * because these methods return a `Self` and this implicit conversion re-wraps that in a + * `VertexPartitionBaseOps`. This relies on the context bound on `Self`. + */ + private implicit def toOps[VD2: ClassTag]( + partition: Self[VD2]): VertexPartitionBaseOps[VD2, Self] = { + implicitly[VertexPartitionBaseOpsConstructor[Self]].toOps(partition) + } +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala index d901d4fe225fe..069e042ed94a3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -55,6 +55,7 @@ object Analytics extends Logging { val conf = new SparkConf() .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + .set("spark.locality.wait", "100000") taskType match { case "pagerank" => @@ -62,12 +63,14 @@ object Analytics extends Logging { var outFname = "" var numEPart = 4 var partitionStrategy: Option[PartitionStrategy] = None + var numIterOpt: Option[Int] = None options.foreach{ case ("tol", v) => tol = v.toFloat case ("output", v) => outFname = v case ("numEPart", v) => numEPart = v.toInt case ("partStrategy", v) => partitionStrategy = Some(pickPartitioner(v)) + case ("numIter", v) => numIterOpt = Some(v.toInt) case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -84,7 +87,10 @@ object Analytics extends Logging { println("GRAPHX: Number of vertices " + graph.vertices.count) println("GRAPHX: Number of edges " + graph.edges.count) - val pr = graph.pageRank(tol).vertices.cache() + val pr = (numIterOpt match { + case Some(numIter) => PageRank.run(graph, numIter) + case None => PageRank.runUntilConvergence(graph, tol) + }).vertices.cache() println("GRAPHX: Total rank: " + pr.map(_._2).reduce(_ + _)) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 32b5fe4813594..7b9bac5d9c8ea 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -110,7 +110,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val p = 100 val verts = 1 to n val graph = Graph.fromEdgeTuples(sc.parallelize(verts.flatMap(x => - verts.filter(y => y % x == 0).map(y => (x: VertexId, y: VertexId))), p), 0) + verts.withFilter(y => y % x == 0).map(y => (x: VertexId, y: VertexId))), p), 0) assert(graph.edges.partitions.length === p) val partitionedGraph = graph.partitionBy(EdgePartition2D) assert(graph.edges.partitions.length === p) @@ -120,7 +120,13 @@ class GraphSuite extends FunSuite with LocalSparkContext { val part = iter.next()._2 Iterator((part.srcIds ++ part.dstIds).toSet) }.collect - assert(verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) + if (!verts.forall(id => partitionSets.count(_.contains(id)) <= bound)) { + val numFailures = verts.count(id => partitionSets.count(_.contains(id)) > bound) + val failure = verts.maxBy(id => partitionSets.count(_.contains(id))) + fail(("Replication bound test failed for %d/%d vertices. " + + "Example: vertex %d replicated to %d (> %f) partitions.").format( + numFailures, n, failure, partitionSets.count(_.contains(failure)), bound)) + } // This should not be true for the default hash partitioning val partitionSetsUnpartitioned = graph.edges.partitionsRDD.mapPartitions { iter => val part = iter.next()._2 diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index e135d1d7ad6a3..d2e0c01bc35ef 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -26,10 +26,16 @@ import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { + def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A, Int] = { + val builder = new EdgePartitionBuilder[A, Int] + for ((src, dst, attr) <- xs) { builder.add(src: VertexId, dst: VertexId, attr) } + builder.toEdgePartition + } + test("reverse") { val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) val reversedEdges = List(Edge(0, 2, 0), Edge(1, 0, 0), Edge(2, 1, 0)) - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- edges) { builder.add(e.srcId, e.dstId, e.attr) } @@ -40,7 +46,7 @@ class EdgePartitionSuite extends FunSuite { test("map") { val edges = List(Edge(0, 1, 0), Edge(1, 2, 0), Edge(2, 0, 0)) - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- edges) { builder.add(e.srcId, e.dstId, e.attr) } @@ -49,11 +55,22 @@ class EdgePartitionSuite extends FunSuite { edges.map(e => e.copy(attr = e.srcId + e.dstId))) } + test("filter") { + val edges = List(Edge(0, 1, 0), Edge(0, 2, 0), Edge(2, 0, 0)) + val builder = new EdgePartitionBuilder[Int, Int] + for (e <- edges) { + builder.add(e.srcId, e.dstId, e.attr) + } + val edgePartition = builder.toEdgePartition + val filtered = edgePartition.filter(et => et.srcId == 0, (vid, attr) => vid == 0 || vid == 1) + assert(filtered.tripletIterator().toList.map(et => (et.srcId, et.dstId)) === List((0L, 1L))) + } + test("groupEdges") { val edges = List( Edge(0, 1, 1), Edge(1, 2, 2), Edge(2, 0, 4), Edge(0, 1, 8), Edge(1, 2, 16), Edge(2, 0, 32)) val groupedEdges = List(Edge(0, 1, 9), Edge(1, 2, 18), Edge(2, 0, 36)) - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- edges) { builder.add(e.srcId, e.dstId, e.attr) } @@ -61,11 +78,19 @@ class EdgePartitionSuite extends FunSuite { assert(edgePartition.groupEdges(_ + _).iterator.map(_.copy()).toList === groupedEdges) } + test("upgradeIterator") { + val edges = List((0, 1, 0), (1, 0, 0)) + val verts = List((0L, 1), (1L, 2)) + val part = makeEdgePartition(edges).updateVertices(verts.iterator) + assert(part.upgradeIterator(part.iterator).map(_.toTuple).toList === + part.tripletIterator().toList.map(_.toTuple)) + } + test("indexIterator") { val edgesFrom0 = List(Edge(0, 1, 0)) val edgesFrom1 = List(Edge(1, 0, 0), Edge(1, 2, 0)) val sortedEdges = edgesFrom0 ++ edgesFrom1 - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Nothing] for (e <- Random.shuffle(sortedEdges)) { builder.add(e.srcId, e.dstId, e.attr) } @@ -77,11 +102,6 @@ class EdgePartitionSuite extends FunSuite { } test("innerJoin") { - def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = { - val builder = new EdgePartitionBuilder[A] - for ((src, dst, attr) <- xs) { builder.add(src: VertexId, dst: VertexId, attr) } - builder.toEdgePartition - } val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) val bList = List((0, 1, 0), (1, 0, 0), (1, 1, 0), (3, 4, 0), (5, 5, 0)) val a = makeEdgePartition(aList) @@ -90,4 +110,14 @@ class EdgePartitionSuite extends FunSuite { assert(a.innerJoin(b) { (src, dst, a, b) => a }.iterator.map(_.copy()).toList === List(Edge(0, 1, 0), Edge(1, 0, 0), Edge(5, 5, 0))) } + + test("isActive, numActives, replaceActives") { + val ep = new EdgePartitionBuilder[Nothing, Nothing].toEdgePartition + .withActiveSet(Iterator(0L, 2L, 0L)) + assert(ep.isActive(0)) + assert(!ep.isActive(1)) + assert(ep.isActive(2)) + assert(!ep.isActive(-1)) + assert(ep.numActives == Some(2)) + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala index 9cbb2d2acdc2d..49b2704390fea 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala @@ -26,17 +26,11 @@ import org.apache.spark.graphx._ class EdgeTripletIteratorSuite extends FunSuite { test("iterator.toList") { - val builder = new EdgePartitionBuilder[Int] + val builder = new EdgePartitionBuilder[Int, Int] builder.add(1, 2, 0) builder.add(1, 3, 0) builder.add(1, 4, 0) - val vidmap = new VertexIdToIndexMap - vidmap.add(1) - vidmap.add(2) - vidmap.add(3) - vidmap.add(4) - val vs = Array.fill(vidmap.capacity)(0) - val iter = new EdgeTripletIterator[Int, Int](vidmap, vs, builder.toEdgePartition) + val iter = new EdgeTripletIterator[Int, Int](builder.toEdgePartition, true, true) val result = iter.toList.map(et => (et.srcId, et.dstId)) assert(result === Seq((1, 2), (1, 3), (1, 4))) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala index a048d13fd12b8..8bf1384d514c1 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala @@ -30,17 +30,6 @@ class VertexPartitionSuite extends FunSuite { assert(!vp.isDefined(-1)) } - test("isActive, numActives, replaceActives") { - val vp = VertexPartition(Iterator((0L, 1), (1L, 1))) - .filter { (vid, attr) => vid == 0 } - .replaceActives(Iterator(0, 2, 0)) - assert(vp.isActive(0)) - assert(!vp.isActive(1)) - assert(vp.isActive(2)) - assert(!vp.isActive(-1)) - assert(vp.numActives == Some(2)) - } - test("map") { val vp = VertexPartition(Iterator((0L, 1), (1L, 1))).map { (vid, attr) => 2 } assert(vp(0) === 2) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index efdb38e907d14..fafc9b36a77d3 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -76,6 +76,8 @@ object MimaBuild { excludeSparkClass("util.XORShiftRandom") ++ excludeSparkClass("graphx.EdgeRDD") ++ excludeSparkClass("graphx.VertexRDD") ++ + excludeSparkClass("graphx.impl.GraphImpl") ++ + excludeSparkClass("graphx.impl.RoutingTable") ++ excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ From 2b7bd29eb6ee5baf739eec143044ecfc296b9b1f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 10 May 2014 20:50:40 -0700 Subject: [PATCH 11/82] SPARK-1789. Multiple versions of Netty dependencies cause FlumeStreamSuite failure TL;DR is there is a bit of JAR hell trouble with Netty, that can be mostly resolved and will resolve a test failure. I hit the error described at http://apache-spark-user-list.1001560.n3.nabble.com/SparkContext-startup-time-out-td1753.html while running FlumeStreamingSuite, and have for a short while (is it just me?) velvia notes: "I have found a workaround. If you add akka 2.2.4 to your dependencies, then everything works, probably because akka 2.2.4 brings in newer version of Jetty." There are at least 3 versions of Netty in play in the build: - the new Flume 1.4.0 dependency brings in io.netty:netty:3.4.0.Final, and that is the immediate problem - the custom version of akka 2.2.3 depends on io.netty:netty:3.6.6. - but, Spark Core directly uses io.netty:netty-all:4.0.17.Final The POMs try to exclude other versions of netty, but are excluding org.jboss.netty:netty, when in fact older versions of io.netty:netty (not netty-all) are also an issue. The org.jboss.netty:netty excludes are largely unnecessary. I replaced many of them with io.netty:netty exclusions until everything agreed on io.netty:netty-all:4.0.17.Final. But this didn't work, since Akka 2.2.3 doesn't work with Netty 4.x. Down-grading to 3.6.6.Final across the board made some Spark code not compile. If the build *keeps* io.netty:netty:3.6.6.Final as well, everything seems to work. Part of the reason seems to be that Netty 3.x used the old `org.jboss.netty` packages. This is less than ideal, but is no worse than the current situation. So this PR resolves the issue and improves the JAR hell, even if it leaves the existing theoretical Netty 3-vs-4 conflict: - Remove org.jboss.netty excludes where possible, for clarity; they're not needed except with Hadoop artifacts - Add io.netty:netty excludes where needed -- except, let akka keep its io.netty:netty - Change a bit of test code that actually depended on Netty 3.x, to use 4.x equivalent - Update SBT build accordingly A better change would be to update Akka far enough such that it agrees on Netty 4.x, but I don't know if that's feasible. Author: Sean Owen Closes #723 from srowen/SPARK-1789 and squashes the following commits: 43661b7 [Sean Owen] Update and add Netty excludes to prevent some JAR conflicts that cause test issues --- .../org/apache/spark/LocalSparkContext.scala | 3 +- examples/pom.xml | 4 +++ external/flume/pom.xml | 2 +- external/mqtt/pom.xml | 6 ---- external/twitter/pom.xml | 6 ---- external/zeromq/pom.xml | 6 ---- pom.xml | 32 ----------------- project/SparkBuild.scala | 35 ++++++++++--------- 8 files changed, 24 insertions(+), 70 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 4b972f88a9542..53e367a61715b 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -17,8 +17,7 @@ package org.apache.spark -import org.jboss.netty.logging.InternalLoggerFactory -import org.jboss.netty.logging.Slf4JLoggerFactory +import _root_.io.netty.util.internal.logging.{Slf4JLoggerFactory, InternalLoggerFactory} import org.scalatest.BeforeAndAfterAll import org.scalatest.BeforeAndAfterEach import org.scalatest.Suite diff --git a/examples/pom.xml b/examples/pom.xml index e1fc149d87f17..874bcd7916f35 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -106,6 +106,10 @@ org.jboss.netty netty + + io.netty + netty + commons-logging commons-logging diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 03d3b2394f510..6aec215687fe0 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -50,7 +50,7 @@ 1.4.0 - org.jboss.netty + io.netty netty diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9aa1c1a9f5b80..7b2dc5ba1d7f9 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -53,12 +53,6 @@ ${akka.group} akka-zeromq_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - org.scalatest diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index a443459594710..5766d3a0d44ec 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -48,12 +48,6 @@ org.twitter4j twitter4j-stream 3.0.3 - - - org.jboss.netty - netty - - org.scalatest diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index a40e55876e640..4ed4196bd8662 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -48,12 +48,6 @@ ${akka.group} akka-zeromq_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - org.scalatest diff --git a/pom.xml b/pom.xml index e0bff60a54cde..c4e1c6be52a1b 100644 --- a/pom.xml +++ b/pom.xml @@ -324,45 +324,21 @@ ${akka.group} akka-actor_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - ${akka.group} akka-remote_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - ${akka.group} akka-slf4j_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - ${akka.group} akka-testkit_${scala.binary.version} ${akka.version} - - - org.jboss.netty - netty - - colt @@ -513,10 +489,6 @@ avro ${avro.version} - - org.jboss.netty - netty - io.netty netty @@ -551,10 +523,6 @@ avro-mapred ${avro.version} - - org.jboss.netty - netty - io.netty netty diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index af882b3ea7beb..a12c61853e410 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -303,7 +303,8 @@ object SparkBuild extends Build { val parquetVersion = "1.4.3" val slf4jVersion = "1.7.5" - val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeJBossNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeIONetty = ExclusionRule(organization = "io.netty") val excludeEclipseJetty = ExclusionRule(organization = "org.eclipse.jetty") val excludeAsm = ExclusionRule(organization = "org.ow2.asm") val excludeOldAsm = ExclusionRule(organization = "asm") @@ -337,8 +338,8 @@ object SparkBuild extends Build { "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 "com.ning" % "compress-lzf" % "1.0.0", "org.xerial.snappy" % "snappy-java" % "1.0.5", - "org.spark-project.akka" %% "akka-remote" % akkaVersion excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-slf4j" % akkaVersion excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-remote" % akkaVersion, + "org.spark-project.akka" %% "akka-slf4j" % akkaVersion, "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "colt" % "colt" % "1.2.0", @@ -346,8 +347,8 @@ object SparkBuild extends Build { "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), - "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), + "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeJBossNetty), "com.codahale.metrics" % "metrics-core" % codahaleMetricsVersion, "com.codahale.metrics" % "metrics-jvm" % codahaleMetricsVersion, "com.codahale.metrics" % "metrics-json" % codahaleMetricsVersion, @@ -421,7 +422,7 @@ object SparkBuild extends Build { v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", - "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), + "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeIONetty, excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), "org.apache.cassandra" % "cassandra-all" % "1.2.6" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") @@ -429,7 +430,7 @@ object SparkBuild extends Build { exclude("io.netty", "netty") exclude("jline","jline") exclude("org.apache.cassandra.deps", "avro") - excludeAll(excludeSLF4J), + excludeAll(excludeSLF4J, excludeIONetty), "com.github.scopt" %% "scopt" % "3.2.0" ) ) ++ assemblySettings ++ extraAssemblySettings @@ -561,11 +562,11 @@ object SparkBuild extends Build { def yarnEnabledSettings = Seq( libraryDependencies ++= Seq( // Exclude rule required for all ? - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm) + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm) ) ) @@ -593,7 +594,7 @@ object SparkBuild extends Build { name := "spark-streaming-twitter", previousArtifact := sparkPreviousArtifact("spark-streaming-twitter"), libraryDependencies ++= Seq( - "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty) + "org.twitter4j" % "twitter4j-stream" % "3.0.3" ) ) @@ -601,12 +602,12 @@ object SparkBuild extends Build { name := "spark-streaming-kafka", previousArtifact := sparkPreviousArtifact("spark-streaming-kafka"), libraryDependencies ++= Seq( - "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), + "com.github.sgroschupf" % "zkclient" % "0.1", "org.apache.kafka" %% "kafka" % "0.8.0" exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") exclude("net.sf.jopt-simple", "jopt-simple") - excludeAll(excludeNetty, excludeSLF4J) + excludeAll(excludeSLF4J) ) ) @@ -614,7 +615,7 @@ object SparkBuild extends Build { name := "spark-streaming-flume", previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeNetty, excludeThrift) + "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeIONetty, excludeThrift) ) ) @@ -622,7 +623,7 @@ object SparkBuild extends Build { name := "spark-streaming-zeromq", previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), libraryDependencies ++= Seq( - "org.spark-project.akka" %% "akka-zeromq" % akkaVersion excludeAll(excludeNetty) + "org.spark-project.akka" %% "akka-zeromq" % akkaVersion ) ) From 83e0424d87022e7a967088365931a08aa06ffd9f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 10 May 2014 20:58:02 -0700 Subject: [PATCH 12/82] [SPARK-1774] Respect SparkSubmit --jars on YARN (client) SparkSubmit ignores `--jars` for YARN client. This is a bug. This PR also automatically adds the application jar to `spark.jar`. Previously, when running as yarn-client, you must specify the jar additionally through `--files` (because `--jars` didn't work). Now you don't have to explicitly specify it through either. Tested on a YARN cluster. Author: Andrew Or Closes #710 from andrewor14/yarn-jars and squashes the following commits: 35d1928 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-jars c27bf6c [Andrew Or] For yarn-cluster and python, do not add primaryResource to spark.jar c92c5bf [Andrew Or] Minor cleanups 269f9f3 [Andrew Or] Fix format 013d840 [Andrew Or] Fix tests 1407474 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-jars 3bb75e8 [Andrew Or] Allow SparkSubmit --jars to take effect in yarn-client mode --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 39 ++++--- .../spark/deploy/SparkSubmitSuite.scala | 110 ++++++++++++------ .../spark/deploy/yarn/ClientArguments.scala | 4 +- 4 files changed, 102 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c639b3e15ded5..71bab295442fc 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -917,7 +917,7 @@ class SparkContext(config: SparkConf) extends Logging { if (SparkHadoopUtil.get.isYarnMode() && (master == "yarn-standalone" || master == "yarn-cluster")) { // In order for this to work in yarn-cluster mode the user must specify the - // --addjars option to the client to upload the file into the distributed cache + // --addJars option to the client to upload the file into the distributed cache // of the AM to make it show up in the current working directory. val fileName = new Path(uri.getPath).getName() try { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 16de6f7cdb100..c6d3cbd2e728b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -67,8 +67,7 @@ object SparkSubmit { private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) /** - * @return - * a tuple containing the arguments for the child, a list of classpath + * @return a tuple containing the arguments for the child, a list of classpath * entries for the child, a list of system propertes, a list of env vars * and the main class for the child */ @@ -115,13 +114,16 @@ object SparkSubmit { val sysProps = new HashMap[String, String]() var childMainClass = "" + val isPython = args.isPython + val isYarnCluster = clusterManager == YARN && deployOnCluster + if (clusterManager == MESOS && deployOnCluster) { printErrorAndExit("Cannot currently run driver on the cluster in Mesos") } // If we're running a Python app, set the Java class to run to be our PythonRunner, add // Python files to deployment list, and pass the main file and Python path to PythonRunner - if (args.isPython) { + if (isPython) { if (deployOnCluster) { printErrorAndExit("Cannot currently run Python driver programs on cluster") } @@ -161,6 +163,7 @@ object SparkSubmit { val options = List[OptionAssigner]( OptionAssigner(args.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), OptionAssigner(args.name, ALL_CLUSTER_MGRS, false, sysProp = "spark.app.name"), + OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"), OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraClassPath"), OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, true, @@ -168,7 +171,8 @@ object SparkSubmit { OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraLibraryPath"), OptionAssigner(args.driverMemory, YARN, true, clOption = "--driver-memory"), - OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"), + OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), + OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), OptionAssigner(args.queue, YARN, true, clOption = "--queue"), OptionAssigner(args.queue, YARN, false, sysProp = "spark.yarn.queue"), OptionAssigner(args.numExecutors, YARN, true, clOption = "--num-executors"), @@ -176,20 +180,18 @@ object SparkSubmit { OptionAssigner(args.executorMemory, YARN, true, clOption = "--executor-memory"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, false, sysProp = "spark.executor.memory"), - OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), - OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), OptionAssigner(args.executorCores, YARN, true, clOption = "--executor-cores"), OptionAssigner(args.executorCores, YARN, false, sysProp = "spark.executor.cores"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, false, sysProp = "spark.cores.max"), OptionAssigner(args.files, YARN, false, sysProp = "spark.yarn.dist.files"), OptionAssigner(args.files, YARN, true, clOption = "--files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), OptionAssigner(args.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), OptionAssigner(args.archives, YARN, true, clOption = "--archives"), OptionAssigner(args.jars, YARN, true, clOption = "--addJars"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), - OptionAssigner(args.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars") + OptionAssigner(args.jars, ALL_CLUSTER_MGRS, false, sysProp = "spark.jars") ) // For client mode make any added jars immediately visible on the classpath @@ -212,9 +214,10 @@ object SparkSubmit { } } - // For standalone mode, add the application jar automatically so the user doesn't have to - // call sc.addJar. TODO: Standalone mode in the cluster - if (clusterManager == STANDALONE) { + // Add the application jar automatically so the user doesn't have to call sc.addJar + // For YARN cluster mode, the jar is already distributed on each node as "app.jar" + // For python files, the primary resource is already distributed as a regular file + if (!isYarnCluster && !isPython) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) if (args.primaryResource != RESERVED_JAR_NAME) { jars = jars ++ Seq(args.primaryResource) @@ -222,11 +225,11 @@ object SparkSubmit { sysProps.put("spark.jars", jars.mkString(",")) } + // Standalone cluster specific configurations if (deployOnCluster && clusterManager == STANDALONE) { if (args.supervise) { childArgs += "--supervise" } - childMainClass = "org.apache.spark.deploy.Client" childArgs += "launch" childArgs += (args.master, args.primaryResource, args.mainClass) @@ -243,6 +246,7 @@ object SparkSubmit { } } + // Read from default spark properties, if any for ((k, v) <- args.getDefaultSparkProperties) { if (!sysProps.contains(k)) sysProps(k) = v } @@ -250,9 +254,12 @@ object SparkSubmit { (childArgs, childClasspath, sysProps, childMainClass) } - private def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], - sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) - { + private def launch( + childArgs: ArrayBuffer[String], + childClasspath: ArrayBuffer[String], + sysProps: Map[String, String], + childMainClass: String, + verbose: Boolean = false) { if (verbose) { printStream.println(s"Main class:\n$childMainClass") printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index c9edb03cdeb0f..6c0deede53784 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -87,25 +87,41 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles arguments with --key=val") { - val clArgs = Seq("--jars=one.jar,two.jar,three.jar", "--name=myApp") + val clArgs = Seq( + "--jars=one.jar,two.jar,three.jar", + "--name=myApp") val appArgs = new SparkSubmitArguments(clArgs) appArgs.jars should be ("one.jar,two.jar,three.jar") appArgs.name should be ("myApp") } test("handles arguments to user program") { - val clArgs = Seq("--name", "myApp", "--class", "Foo", "userjar.jar", "some", "--weird", "args") + val clArgs = Seq( + "--name", "myApp", + "--class", "Foo", + "userjar.jar", + "some", + "--weird", "args") val appArgs = new SparkSubmitArguments(clArgs) appArgs.childArgs should be (Seq("some", "--weird", "args")) } test("handles YARN cluster mode") { - val clArgs = Seq("--deploy-mode", "cluster", - "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", - "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "beauty", - "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "cluster", + "--master", "yarn", + "--executor-memory", "5g", + "--executor-cores", "5", + "--class", "org.SomeClass", + "--jars", "one.jar,two.jar,three.jar", + "--driver-memory", "4g", + "--queue", "thequeue", + "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", + "--num-executors", "6", + "--name", "beauty", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -127,12 +143,21 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles YARN client mode") { - val clArgs = Seq("--deploy-mode", "client", - "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", - "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "trill", - "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "yarn", + "--executor-memory", "5g", + "--executor-cores", "5", + "--class", "org.SomeClass", + "--jars", "one.jar,two.jar,three.jar", + "--driver-memory", "4g", + "--queue", "thequeue", + "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", + "--num-executors", "6", + "--name", "trill", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -142,6 +167,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { classpath should contain ("two.jar") classpath should contain ("three.jar") sysProps("spark.app.name") should be ("trill") + sysProps("spark.jars") should be ("one.jar,two.jar,three.jar,thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.executor.cores") should be ("5") sysProps("spark.yarn.queue") should be ("thequeue") @@ -152,9 +178,15 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone cluster mode") { - val clArgs = Seq("--deploy-mode", "cluster", - "--master", "spark://h:p", "--class", "org.SomeClass", - "--supervise", "--driver-memory", "4g", "--driver-cores", "5", "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "cluster", + "--master", "spark://h:p", + "--class", "org.SomeClass", + "--supervise", + "--driver-memory", "4g", + "--driver-cores", "5", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -166,9 +198,15 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone client mode") { - val clArgs = Seq("--deploy-mode", "client", - "--master", "spark://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "spark://h:p", + "--executor-memory", "5g", + "--total-executor-cores", "5", + "--class", "org.SomeClass", + "--driver-memory", "4g", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -179,9 +217,15 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles mesos client mode") { - val clArgs = Seq("--deploy-mode", "client", - "--master", "mesos://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") + val clArgs = Seq( + "--deploy-mode", "client", + "--master", "mesos://h:p", + "--executor-memory", "5g", + "--total-executor-cores", "5", + "--class", "org.SomeClass", + "--driver-memory", "4g", + "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -192,15 +236,17 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("launch simple application with spark-submit") { - runSparkSubmit( - Seq( - "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), - "--name", "testApp", - "--master", "local", - "unUsed.jar")) + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + unusedJar.toString) + runSparkSubmit(args) } test("spark submit includes jars passed in through --jar") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") @@ -209,7 +255,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { "--name", "testApp", "--master", "local-cluster[2,1,512]", "--jars", jarsString, - "unused.jar") + unusedJar.toString) runSparkSubmit(args) } @@ -227,7 +273,7 @@ object JarCreationTest { def main(args: Array[String]) { val conf = new SparkConf() val sc = new SparkContext(conf) - val result = sc.makeRDD(1 to 100, 10).mapPartitions{ x => + val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => var foundClasses = false try { Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) @@ -248,7 +294,6 @@ object SimpleApplicationTest { def main(args: Array[String]) { val conf = new SparkConf() val sc = new SparkContext(conf) - val configs = Seq("spark.master", "spark.app.name") for (config <- configs) { val masterValue = conf.get(config) @@ -266,6 +311,5 @@ object SimpleApplicationTest { s"Master had $config=$masterValue but executor had $config=$executorValue") } } - } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 3e4c739e34fe9..b2c413b6d267c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.yarn import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.SparkConf -import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} +import org.apache.spark.scheduler.InputFormatInfo import org.apache.spark.util.IntParam import org.apache.spark.util.MemoryParam @@ -40,9 +40,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" - // TODO var inputFormatInfo: List[InputFormatInfo] = null - // TODO(harvey) var priority = 0 parseArgs(args.toList) From 70bcdef48a051028598d380d41dfce1c9bfb2b9b Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 10 May 2014 21:08:04 -0700 Subject: [PATCH 13/82] Enabled incremental build that comes with sbt 0.13.2 More info at. https://github.com/sbt/sbt/issues/1010 Author: Prashant Sharma Closes #525 from ScrapCodes/sbt-inc-opt and squashes the following commits: ba8fa42 [Prashant Sharma] Enabled incremental build that comes with sbt 0.13.2 --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a12c61853e410..12791e490ae2b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -176,7 +176,7 @@ object SparkBuild extends Build { retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), - + incOptions := incOptions.value.withNameHashing(true), // Fork new JVMs for tests and set Java options for those fork := true, javaOptions in Test += "-Dspark.home=" + sparkHome, From 6bee01dd04ef73c6b829110ebcdd622d521ea8ff Mon Sep 17 00:00:00 2001 From: witgo Date: Sun, 11 May 2014 14:34:27 -0700 Subject: [PATCH 14/82] remove outdated runtime Information scala home Author: witgo Closes #728 from witgo/scala_home and squashes the following commits: cdfd8be [witgo] Merge branch 'master' of https://github.com/apache/spark into scala_home fac094a [witgo] remove outdated runtime Information scala home --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 806e77d98fc5f..19d507c0cf860 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -281,8 +281,7 @@ object SparkEnv extends Logging { val jvmInformation = Seq( ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) + ("Scala Version", Properties.versionString) ).sorted // Spark properties From 7d9cc9214bd06495f6838e355331dd2b5f1f7407 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 11 May 2014 17:11:55 -0700 Subject: [PATCH 15/82] SPARK-1770: Load balance elements when repartitioning. This patch adds better balancing when performing a repartition of an RDD. Previously the elements in the RDD were hash partitioned, meaning if the RDD was skewed certain partitions would end up being very large. This commit adds load balancing of elements across the repartitioned RDD splits. The load balancing is not perfect: a given output partition can have up to N more elements than the average if there are N input partitions. However, some randomization is used to minimize the probabiliy that this happens. Author: Patrick Wendell Closes #727 from pwendell/load-balance and squashes the following commits: f9da752 [Patrick Wendell] Response to Matei's feedback acfa46a [Patrick Wendell] SPARK-1770: Load balance elements when repartitioning. --- .../main/scala/org/apache/spark/rdd/RDD.scala | 15 +++++++-- .../scala/org/apache/spark/rdd/RDDSuite.scala | 33 +++++++++++++++++++ 2 files changed, 46 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index a1ca612cc9a09..aa03e9276fb34 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -328,11 +328,22 @@ abstract class RDD[T: ClassTag]( def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) : RDD[T] = { if (shuffle) { + /** Distributes elements evenly across output partitions, starting from a random partition. */ + def distributePartition(index: Int, items: Iterator[T]): Iterator[(Int, T)] = { + var position = (new Random(index)).nextInt(numPartitions) + items.map { t => + // Note that the hash code of the key will just be the key itself. The HashPartitioner + // will mod it with the number of total partitions. + position = position + 1 + (position, t) + } + } + // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), + new ShuffledRDD[Int, T, (Int, T)](mapPartitionsWithIndex(distributePartition), new HashPartitioner(numPartitions)), - numPartitions).keys + numPartitions).values } else { new CoalescedRDD(this, numPartitions) } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 8da9a0da700e0..e686068f7a99a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -202,6 +202,39 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(repartitioned2.collect().toSet === (1 to 1000).toSet) } + test("repartitioned RDDs perform load balancing") { + // Coalesce partitions + val input = Array.fill(1000)(1) + val initialPartitions = 10 + val data = sc.parallelize(input, initialPartitions) + + val repartitioned1 = data.repartition(2) + assert(repartitioned1.partitions.size == 2) + val partitions1 = repartitioned1.glom().collect() + // some noise in balancing is allowed due to randomization + assert(math.abs(partitions1(0).length - 500) < initialPartitions) + assert(math.abs(partitions1(1).length - 500) < initialPartitions) + assert(repartitioned1.collect() === input) + + def testSplitPartitions(input: Seq[Int], initialPartitions: Int, finalPartitions: Int) { + val data = sc.parallelize(input, initialPartitions) + val repartitioned = data.repartition(finalPartitions) + assert(repartitioned.partitions.size === finalPartitions) + val partitions = repartitioned.glom().collect() + // assert all elements are present + assert(repartitioned.collect().sortWith(_ > _).toSeq === input.toSeq.sortWith(_ > _).toSeq) + // assert no bucket is overloaded + for (partition <- partitions) { + val avg = input.size / finalPartitions + val maxPossible = avg + initialPartitions + assert(partition.length <= maxPossible) + } + } + + testSplitPartitions(Array.fill(100)(1), 10, 20) + testSplitPartitions(Array.fill(10000)(1) ++ Array.fill(10000)(2), 20, 100) + } + test("coalesced RDDs") { val data = sc.parallelize(1 to 10, 10) From 05c9aa9eb1b7f13cd40bbca23e6bc7e1d20e91cd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 11 May 2014 18:17:34 -0700 Subject: [PATCH 16/82] SPARK-1652: Set driver memory correctly in spark-submit. The previous check didn't account for the fact that the default deploy mode is "client" unless otherwise specified. Also, this sets the more narrowly defined SPARK_DRIVER_MEMORY instead of setting SPARK_MEM. Author: Patrick Wendell Closes #730 from pwendell/spark-submit and squashes the following commits: 430b98f [Patrick Wendell] Feedback from Aaron e788edf [Patrick Wendell] Changes based on Aaron's feedback f508146 [Patrick Wendell] SPARK-1652: Set driver memory correctly in spark-submit. --- bin/spark-submit | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/bin/spark-submit b/bin/spark-submit index 49bc26252cadf..63903b17a2902 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -35,8 +35,10 @@ while (($#)); do shift done -if [ ! -z $DRIVER_MEMORY ] && [ ! -z $DEPLOY_MODE ] && [ $DEPLOY_MODE = "client" ]; then - export SPARK_MEM=$DRIVER_MEMORY +DEPLOY_MODE=${DEPLOY_MODE:-"client"} + +if [ -n "$DRIVER_MEMORY" ] && [ $DEPLOY_MODE == "client" ]; then + export SPARK_DRIVER_MEMORY=$DRIVER_MEMORY fi $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" From f938a155b2a9c126b292d5403aca31de83d5105a Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 11 May 2014 18:33:46 -0700 Subject: [PATCH 17/82] Fix error in 2d Graph Partitioner Their was a minor bug in which negative partition ids could be generated when constructing a 2D partitioning of a graph. This could lead to an inefficient 2D partition for large vertex id values. Author: Joseph E. Gonzalez Closes #709 from jegonzal/fix_2d_partitioning and squashes the following commits: 937c562 [Joseph E. Gonzalez] fixing bug in 2d partitioning algorithm where negative partition ids could be generated. --- .../scala/org/apache/spark/graphx/PartitionStrategy.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 0470d74cf9efe..1526ccef06fd4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -78,8 +78,8 @@ object PartitionStrategy { override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = { val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt val mixingPrime: VertexId = 1125899906842597L - val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt - val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt + val col: PartitionID = (math.abs(src * mixingPrime) % ceilSqrtNumParts).toInt + val row: PartitionID = (math.abs(dst * mixingPrime) % ceilSqrtNumParts).toInt (col * ceilSqrtNumParts + row) % numParts } } From a6b02fb7486356493474c7f42bb714c9cce215ca Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Sun, 11 May 2014 19:20:42 -0700 Subject: [PATCH 18/82] SPARK-1786: Edge Partition Serialization This appears to address the issue with edge partition serialization. The solution appears to be just registering the `PrimitiveKeyOpenHashMap`. However I noticed that we appear to have forked that code in GraphX but retained the same name (which is confusing). I also renamed our local copy to `GraphXPrimitiveKeyOpenHashMap`. We should consider dropping that and using the one in Spark if possible. Author: Ankur Dave Author: Joseph E. Gonzalez Closes #724 from jegonzal/edge_partition_serialization and squashes the following commits: b0a525a [Ankur Dave] Disable reference tracking to fix serialization test bb7f548 [Ankur Dave] Add failing test for EdgePartition Kryo serialization 67dac22 [Joseph E. Gonzalez] Making EdgePartition serializable. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 ++++++--- .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...ala => GraphXPrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ++++++++++++++++++ 11 files changed, 44 insertions(+), 23 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{PrimitiveKeyOpenHashMap.scala => GraphXPrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index d295d0127ac72..f97f329c0e832 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,6 +24,9 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.OpenHashSet + /** * Registers GraphX classes with Kryo for improved performance. @@ -43,8 +46,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - - // This avoids a large number of hash table lookups. - kryo.setReferences(false) + kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) + kryo.register(classOf[OpenHashSet[Int]]) + kryo.register(classOf[OpenHashSet[Long]]) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 871e81f8d245c..a5c9cd1f8b4e6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - @transient val srcIds: Array[VertexId], - @transient val dstIds: Array[VertexId], - @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], - @transient val vertices: VertexPartition[VD], - @transient val activeSet: Option[VertexSet] = None + val srcIds: Array[VertexId] = null, + val dstIds: Array[VertexId] = null, + val data: Array[ED] = null, + val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, + val vertices: VertexPartition[VD] = null, + val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index ecb49bef42e45..4520beb991515 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val srcIds = new Array[VertexId](edgeArray.size) val dstIds = new Array[VertexId](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new PrimitiveKeyOpenHashMap[VertexId, Int] + val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index ebb0b9418d65d..56f79a7097fce 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 927e32ad0f448..d02e9238adba5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index f4e221d4e05ae..dca54b8a7da86 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index f1d174720a1ba..55c7a19d1bdab 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 8d9e0204d27f2..34939b24440aa 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 21ff615feca6c..a4f769b294010 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 7b02e2ed1a9cb..57b01b6f2e1fb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index d2e0c01bc35ef..28fd112f2b124 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,6 +22,9 @@ import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoSerializer + import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -120,4 +123,19 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } + + test("Kryo serialization") { + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val a: EdgePartition[Int, Int] = makeEdgePartition(aList) + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + val s = new KryoSerializer(conf).newInstance() + val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) + assert(aSer.srcIds.toList === a.srcIds.toList) + assert(aSer.dstIds.toList === a.dstIds.toList) + assert(aSer.data.toList === a.data.toList) + assert(aSer.index != null) + assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) + } } From af15c82bfe2c3f73142b8f310784a0e85841539d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 10:49:03 -0700 Subject: [PATCH 19/82] Revert "SPARK-1786: Edge Partition Serialization" This reverts commit a6b02fb7486356493474c7f42bb714c9cce215ca. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 +++------ .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...Map.scala => PrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ------------------ 11 files changed, 23 insertions(+), 44 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{GraphXPrimitiveKeyOpenHashMap.scala => PrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index f97f329c0e832..d295d0127ac72 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,9 +24,6 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap -import org.apache.spark.util.collection.OpenHashSet - /** * Registers GraphX classes with Kryo for improved performance. @@ -46,8 +43,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) - kryo.register(classOf[OpenHashSet[Int]]) - kryo.register(classOf[OpenHashSet[Long]]) + + // This avoids a large number of hash table lookups. + kryo.setReferences(false) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index a5c9cd1f8b4e6..871e81f8d245c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - val srcIds: Array[VertexId] = null, - val dstIds: Array[VertexId] = null, - val data: Array[ED] = null, - val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, - val vertices: VertexPartition[VD] = null, - val activeSet: Option[VertexSet] = None + @transient val srcIds: Array[VertexId], + @transient val dstIds: Array[VertexId], + @transient val data: Array[ED], + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], + @transient val vertices: VertexPartition[VD], + @transient val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 4520beb991515..ecb49bef42e45 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val srcIds = new Array[VertexId](edgeArray.size) val dstIds = new Array[VertexId](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] + val index = new PrimitiveKeyOpenHashMap[VertexId, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index 56f79a7097fce..ebb0b9418d65d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index d02e9238adba5..927e32ad0f448 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index dca54b8a7da86..f4e221d4e05ae 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 55c7a19d1bdab..f1d174720a1ba 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 34939b24440aa..8d9e0204d27f2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val map = new PrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index a4f769b294010..21ff615feca6c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala index 57b01b6f2e1fb..7b02e2ed1a9cb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index 28fd112f2b124..d2e0c01bc35ef 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,9 +22,6 @@ import scala.util.Random import org.scalatest.FunSuite -import org.apache.spark.SparkConf -import org.apache.spark.serializer.KryoSerializer - import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -123,19 +120,4 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } - - test("Kryo serialization") { - val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) - val a: EdgePartition[Int, Int] = makeEdgePartition(aList) - val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - val s = new KryoSerializer(conf).newInstance() - val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) - assert(aSer.srcIds.toList === a.srcIds.toList) - assert(aSer.dstIds.toList === a.dstIds.toList) - assert(aSer.data.toList === a.data.toList) - assert(aSer.index != null) - assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) - } } From 3af1f386439cdddd42e545ad63d089f4dfdf9f8a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 12 May 2014 11:08:52 -0700 Subject: [PATCH 20/82] SPARK-1772 Stop catching Throwable, let Executors die The main issue this patch fixes is [SPARK-1772](https://issues.apache.org/jira/browse/SPARK-1772), in which Executors may not die when fatal exceptions (e.g., OOM) are thrown. This patch causes Executors to delegate to the ExecutorUncaughtExceptionHandler when a fatal exception is thrown. This patch also continues the fight in the neverending war against `case t: Throwable =>`, by only catching Exceptions in many places, and adding a wrapper for Threads and Runnables to make sure any uncaught exceptions are at least printed to the logs. It also turns out that it is unlikely that the IndestructibleActorSystem actually works, given testing ([here](https://gist.github.com/aarondav/ca1f0cdcd50727f89c0d)). The uncaughtExceptionHandler is not called from the places that we expected it would be. [SPARK-1620](https://issues.apache.org/jira/browse/SPARK-1620) deals with part of this issue, but refactoring our Actor Systems to ensure that exceptions are dealt with properly is a much bigger change, outside the scope of this PR. Author: Aaron Davidson Closes #715 from aarondav/throwable and squashes the following commits: f9b9bfe [Aaron Davidson] Remove other redundant 'throw e' e937a0a [Aaron Davidson] Address Prashant and Matei's comments 1867867 [Aaron Davidson] [RFC] SPARK-1772 Stop catching Throwable, let Executors die --- .../org/apache/spark/ContextCleaner.scala | 11 +-- .../scala/org/apache/spark/SparkContext.scala | 12 ++-- .../apache/spark/api/python/PythonRDD.scala | 3 +- .../api/python/PythonWorkerFactory.scala | 1 - .../org/apache/spark/deploy/Client.scala | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../spark/deploy/history/HistoryServer.scala | 8 +-- .../apache/spark/deploy/master/Master.scala | 4 +- .../spark/deploy/worker/DriverWrapper.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 37 +++------- .../ExecutorUncaughtExceptionHandler.scala | 53 +++++++++++++++ .../scheduler/EventLoggingListener.scala | 4 +- .../spark/scheduler/TaskResultGetter.scala | 8 +-- .../spark/storage/DiskBlockManager.scala | 6 +- .../spark/storage/TachyonBlockManager.scala | 7 +- .../org/apache/spark/util/AkkaUtils.scala | 11 +-- .../util/IndestructibleActorSystem.scala | 68 ------------------- .../scala/org/apache/spark/util/Utils.scala | 26 ++++++- 19 files changed, 127 insertions(+), 140 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 54e08d7866f75..e2d2250982daa 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * Classes that represent cleaning tasks. @@ -110,7 +111,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Keep cleaning RDD, shuffle, and broadcast state. */ - private def keepCleaning() { + private def keepCleaning(): Unit = Utils.logUncaughtExceptions { while (!stopped) { try { val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) @@ -128,7 +129,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } } catch { - case t: Throwable => logError("Error in cleaning thread", t) + case e: Exception => logError("Error in cleaning thread", e) } } } @@ -141,7 +142,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.rddCleaned(rddId)) logInfo("Cleaned RDD " + rddId) } catch { - case t: Throwable => logError("Error cleaning RDD " + rddId, t) + case e: Exception => logError("Error cleaning RDD " + rddId, e) } } @@ -154,7 +155,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.shuffleCleaned(shuffleId)) logInfo("Cleaned shuffle " + shuffleId) } catch { - case t: Throwable => logError("Error cleaning shuffle " + shuffleId, t) + case e: Exception => logError("Error cleaning shuffle " + shuffleId, e) } } @@ -166,7 +167,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { listeners.foreach(_.broadcastCleaned(broadcastId)) logInfo("Cleaned broadcast " + broadcastId) } catch { - case t: Throwable => logError("Error cleaning broadcast " + broadcastId, t) + case e: Exception => logError("Error cleaning broadcast " + broadcastId, e) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 71bab295442fc..e6121a705497c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1494,8 +1494,8 @@ object SparkContext extends Logging { } catch { // TODO: Enumerate the exact reasons why it can fail // But irrespective of it, it means we cannot proceed ! - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) @@ -1510,8 +1510,8 @@ object SparkContext extends Logging { cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } @@ -1521,8 +1521,8 @@ object SparkContext extends Logging { val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { - case th: Throwable => { - throw new SparkException("YARN mode not available ?", th) + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 2971c277aa863..57b28b9972366 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -171,7 +171,7 @@ private[spark] class PythonRDD[T: ClassTag]( this.interrupt() } - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { SparkEnv.set(env) val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) @@ -282,7 +282,6 @@ private[spark] object PythonRDD { } } catch { case eof: EOFException => {} - case e: Throwable => throw e } JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 002f2acd94dee..759cbe2c46c52 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -71,7 +71,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String stopDaemon() startDaemon() new Socket(daemonHost, daemonPort) - case e: Throwable => throw e } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 7ead1171525d2..aeb159adc31d9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -157,7 +157,7 @@ object Client { // TODO: See if we can initialize akka so return messages are sent back using the same TCP // flow. Else, this (sadly) requires the DriverClient be routable from the Master. val (actorSystem, _) = AkkaUtils.createActorSystem( - "driverClient", Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index e2df1b8954124..148115d3ed351 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -103,7 +103,7 @@ object SparkHadoopUtil { .newInstance() .asInstanceOf[SparkHadoopUtil] } catch { - case th: Throwable => throw new SparkException("Unable to load YARN support", th) + case e: Exception => throw new SparkException("Unable to load YARN support", e) } } else { new SparkHadoopUtil diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 1238bbf9da2fd..a9c11dca5678e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -70,7 +70,7 @@ class HistoryServer( * TODO: Add a mechanism to update manually. */ private val logCheckingThread = new Thread { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { while (!stopped) { val now = System.currentTimeMillis if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { @@ -154,7 +154,7 @@ class HistoryServer( numCompletedApplications = logInfos.size } catch { - case t: Throwable => logError("Exception in checking for event log updates", t) + case e: Exception => logError("Exception in checking for event log updates", e) } } else { logWarning("Attempted to check for event log updates before binding the server.") @@ -231,8 +231,8 @@ class HistoryServer( dir.getModificationTime } } catch { - case t: Throwable => - logError("Exception in accessing modification time of %s".format(dir.getPath), t) + case e: Exception => + logError("Exception in accessing modification time of %s".format(dir.getPath), e) -1L } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index f254f5585ba25..c6dec305bffcb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -684,8 +684,8 @@ private[spark] class Master( webUi.attachSparkUI(ui) return true } catch { - case t: Throwable => - logError("Exception in replaying log for application %s (%s)".format(appName, app.id), t) + case e: Exception => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), e) } } else { logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index be15138f62406..05e242e6df702 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -31,7 +31,7 @@ object DriverWrapper { case workerUrl :: mainClass :: extraArgs => val conf = new SparkConf() val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", - Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) + Utils.localHostName(), 0, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") // Delegate to supplied main class diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index e912ae8a5d3c5..84aec65b7765d 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -105,7 +105,7 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a // SparkEnv / Executor before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = conf, new SecurityManager(conf)) + conf, new SecurityManager(conf)) // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 98e7e0be813be..baee7a216a7c3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -74,28 +74,7 @@ private[spark] class Executor( // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. - Thread.setDefaultUncaughtExceptionHandler( - new Thread.UncaughtExceptionHandler { - override def uncaughtException(thread: Thread, exception: Throwable) { - try { - logError("Uncaught exception in thread " + thread, exception) - - // We may have been called from a shutdown hook. If so, we must not call System.exit(). - // (If we do, we will deadlock.) - if (!Utils.inShutdown()) { - if (exception.isInstanceOf[OutOfMemoryError]) { - System.exit(ExecutorExitCode.OOM) - } else { - System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) - } - } - } catch { - case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) - case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) - } - } - } - ) + Thread.setDefaultUncaughtExceptionHandler(ExecutorUncaughtExceptionHandler) } val executorSource = new ExecutorSource(this, executorId) @@ -259,6 +238,11 @@ private[spark] class Executor( } case t: Throwable => { + // Attempt to exit cleanly by informing the driver of our failure. + // If anything goes wrong (or this was a fatal exception), we will delegate to + // the default uncaught exception handler, which will terminate the Executor. + logError("Exception in task ID " + taskId, t) + val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { @@ -268,10 +252,11 @@ private[spark] class Executor( val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - // TODO: Should we exit the whole executor here? On the one hand, the failed task may - // have left some weird state around depending on when the exception was thrown, but on - // the other hand, maybe we could detect that when future tasks fail and exit then. - logError("Exception in task ID " + taskId, t) + // Don't forcibly exit unless the exception was inherently fatal, to avoid + // stopping other tasks unnecessarily. + if (Utils.isFatalError(t)) { + ExecutorUncaughtExceptionHandler.uncaughtException(t) + } } } finally { // TODO: Unregister shuffle memory only for ResultTask diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala new file mode 100644 index 0000000000000..b0e984c03964c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.Logging +import org.apache.spark.util.Utils + +/** + * The default uncaught exception handler for Executors terminates the whole process, to avoid + * getting into a bad state indefinitely. Since Executors are relatively lightweight, it's better + * to fail fast when things go wrong. + */ +private[spark] object ExecutorUncaughtExceptionHandler + extends Thread.UncaughtExceptionHandler with Logging { + + override def uncaughtException(thread: Thread, exception: Throwable) { + try { + logError("Uncaught exception in thread " + thread, exception) + + // We may have been called from a shutdown hook. If so, we must not call System.exit(). + // (If we do, we will deadlock.) + if (!Utils.inShutdown()) { + if (exception.isInstanceOf[OutOfMemoryError]) { + System.exit(ExecutorExitCode.OOM) + } else { + System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + } + } + } catch { + case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) + case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) + } + } + + def uncaughtException(exception: Throwable) { + uncaughtException(Thread.currentThread(), exception) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 7968a0691db10..a90b0d475c04e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -206,8 +206,8 @@ private[spark] object EventLoggingListener extends Logging { applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } ) } catch { - case t: Throwable => - logError("Exception in parsing logging info from directory %s".format(logDir), t) + case e: Exception => + logError("Exception in parsing logging info from directory %s".format(logDir), e) EventLoggingInfo.empty } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index c9ad2b151daf0..99d305b36a959 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -43,7 +43,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { val result = serializer.get().deserialize[TaskResult[_]](serializedData) match { case directResult: DirectTaskResult[_] => directResult @@ -70,7 +70,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex: Throwable => + case ex: Exception => taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) } } @@ -81,7 +81,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul serializedData: ByteBuffer) { var reason : TaskEndReason = UnknownReason getTaskResultExecutor.execute(new Runnable { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { try { if (serializedData != null && serializedData.limit() > 0) { reason = serializer.get().deserialize[TaskEndReason]( @@ -94,7 +94,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul val loader = Utils.getContextOrSparkClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex: Throwable => {} + case ex: Exception => {} } scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index cf6ef0029a861..3a7243a1ba19c 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -148,7 +148,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD private def addShutdownHook() { localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") DiskBlockManager.this.stop() } @@ -162,8 +162,8 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD try { if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) } catch { - case t: Throwable => - logError("Exception while deleting local spark dir: " + localDir, t) + case e: Exception => + logError("Exception while deleting local spark dir: " + localDir, e) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala index b0b9674856568..a6cbe3aa440ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -25,7 +25,6 @@ import tachyon.client.TachyonFile import org.apache.spark.Logging import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.ShuffleSender import org.apache.spark.util.Utils @@ -137,7 +136,7 @@ private[spark] class TachyonBlockManager( private def addShutdownHook() { tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") tachyonDirs.foreach { tachyonDir => try { @@ -145,8 +144,8 @@ private[spark] class TachyonBlockManager( Utils.deleteRecursively(tachyonDir, client) } } catch { - case t: Throwable => - logError("Exception while deleting tachyon spark dir: " + tachyonDir, t) + case e: Exception => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, e) } } } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 8afe09a117ebc..a8d12bb2a0165 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.duration.{Duration, FiniteDuration} -import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} +import akka.actor.{ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} @@ -41,7 +41,7 @@ private[spark] object AkkaUtils extends Logging { * If indestructible is set to true, the Actor System will continue running in the event * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. */ - def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, + def createActorSystem(name: String, host: String, port: Int, conf: SparkConf, securityManager: SecurityManager): (ActorSystem, Int) = { val akkaThreads = conf.getInt("spark.akka.threads", 4) @@ -101,12 +101,7 @@ private[spark] object AkkaUtils extends Logging { |akka.log-dead-letters-during-shutdown = $lifecycleEvents """.stripMargin)) - val actorSystem = if (indestructible) { - IndestructibleActorSystem(name, akkaConf) - } else { - ActorSystem(name, akkaConf) - } - + val actorSystem = ActorSystem(name, akkaConf) val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get (actorSystem, boundPort) diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala deleted file mode 100644 index 4188a869c13da..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -// Must be in akka.actor package as ActorSystemImpl is protected[akka]. -package akka.actor - -import scala.util.control.{ControlThrowable, NonFatal} - -import com.typesafe.config.Config - -/** - * An akka.actor.ActorSystem which refuses to shut down in the event of a fatal exception - * This is necessary as Spark Executors are allowed to recover from fatal exceptions - * (see org.apache.spark.executor.Executor) - */ -object IndestructibleActorSystem { - def apply(name: String, config: Config): ActorSystem = - apply(name, config, ActorSystem.findClassLoader()) - - def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = - new IndestructibleActorSystemImpl(name, config, classLoader).start() -} - -private[akka] class IndestructibleActorSystemImpl( - override val name: String, - applicationConfig: Config, - classLoader: ClassLoader) - extends ActorSystemImpl(name, applicationConfig, classLoader) { - - protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = { - val fallbackHandler = super.uncaughtExceptionHandler - - new Thread.UncaughtExceptionHandler() { - def uncaughtException(thread: Thread, cause: Throwable): Unit = { - if (isFatalError(cause) && !settings.JvmExitOnFatalError) { - log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + - "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name) - // shutdown() //TODO make it configurable - } else { - fallbackHandler.uncaughtException(thread, cause) - } - } - } - } - - def isFatalError(e: Throwable): Boolean = { - e match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => - false - case _ => - true - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 95777fbf57d8b..8f7594ada2ba1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -29,6 +29,7 @@ import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag import scala.util.Try +import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -41,7 +42,6 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} - /** * Various utility methods used by Spark. */ @@ -1125,4 +1125,28 @@ private[spark] object Utils extends Logging { } } + /** + * Executes the given block, printing and re-throwing any uncaught exceptions. + * This is particularly useful for wrapping code that runs in a thread, to ensure + * that exceptions are printed, and to avoid having to catch Throwable. + */ + def logUncaughtExceptions[T](f: => T): T = { + try { + f + } catch { + case t: Throwable => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + throw t + } + } + + /** Returns true if the given exception was fatal. See docs for scala.util.control.NonFatal. */ + def isFatalError(e: Throwable): Boolean = { + e match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + false + case _ => + true + } + } } From d9c97ba39723d36d3dcb53a75a3a2565357c0955 Mon Sep 17 00:00:00 2001 From: Bernardo Gomez Palacio Date: Mon, 12 May 2014 11:10:28 -0700 Subject: [PATCH 21/82] SPARK-1806: Upgrade Mesos dependency to 0.18.1 Enabled Mesos (0.18.1) dependency with shaded protobuf Why is this needed? Avoids any protobuf version collision between Mesos and any other dependency in Spark e.g. Hadoop HDFS 2.2+ or 1.0.4. Ticket: https://issues.apache.org/jira/browse/SPARK-1806 * Should close https://issues.apache.org/jira/browse/SPARK-1433 Author berngp Author: Bernardo Gomez Palacio Closes #741 from berngp/feature/SPARK-1806 and squashes the following commits: 5d70646 [Bernardo Gomez Palacio] SPARK-1806: Upgrade Mesos dependency to 0.18.1 --- core/pom.xml | 3 ++- .../apache/spark/executor/MesosExecutorBackend.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- pom.xml | 10 +++++++++- project/SparkBuild.scala | 2 +- 5 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 8fe215ab24289..bab50f5ce2888 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -140,6 +140,7 @@ org.apache.mesos mesos + ${mesos.classifier} io.netty @@ -322,7 +323,7 @@ - + src/main/resources diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 9b56f711e0e0b..74100498bb2bd 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import com.google.protobuf.ByteString +import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c975f312324ed..2cea1c8cf154a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -24,7 +24,7 @@ import java.util.Collections import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import com.google.protobuf.ByteString +import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} diff --git a/pom.xml b/pom.xml index c4e1c6be52a1b..dd1d262881b93 100644 --- a/pom.xml +++ b/pom.xml @@ -113,7 +113,8 @@ 2.10.4 2.10 - 0.13.0 + 0.18.1 + shaded-protobuf org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -349,6 +350,13 @@ org.apache.mesos mesos ${mesos.version} + ${mesos.classifier} + + + com.google.protobuf + protobuf-java + + commons-net diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 12791e490ae2b..57b3e22f81713 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -343,7 +343,7 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.13.0", + "org.apache.mesos" % "mesos" % "0.18.1" classifier("shaded-protobuf") exclude("com.google.protobuf", "protobuf-java"), "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From 0e2bde2030f8e455c5a269fc38d4ff05b395ca32 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 12 May 2014 13:05:24 -0700 Subject: [PATCH 22/82] SPARK-1786: Reopening PR 724 Addressing issue in MimaBuild.scala. Author: Ankur Dave Author: Joseph E. Gonzalez Closes #742 from jegonzal/edge_partition_serialization and squashes the following commits: 8ba6e0d [Ankur Dave] Add concatenation operators to MimaBuild.scala cb2ed3a [Joseph E. Gonzalez] addressing missing exclusion in MimaBuild.scala 5d27824 [Ankur Dave] Disable reference tracking to fix serialization test c0a9ae5 [Ankur Dave] Add failing test for EdgePartition Kryo serialization a4a3faa [Joseph E. Gonzalez] Making EdgePartition serializable. --- .../spark/graphx/GraphKryoRegistrator.scala | 9 ++++++--- .../spark/graphx/impl/EdgePartition.scala | 14 +++++++------- .../graphx/impl/EdgePartitionBuilder.scala | 4 ++-- .../graphx/impl/EdgeTripletIterator.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 4 ++-- .../graphx/impl/ShippableVertexPartition.scala | 2 +- .../spark/graphx/impl/VertexPartition.scala | 2 +- .../graphx/impl/VertexPartitionBase.scala | 6 +++--- .../graphx/impl/VertexPartitionBaseOps.scala | 4 ++-- ...ala => GraphXPrimitiveKeyOpenHashMap.scala} | 2 +- .../spark/graphx/impl/EdgePartitionSuite.scala | 18 ++++++++++++++++++ project/MimaBuild.scala | 8 +++++--- 12 files changed, 49 insertions(+), 26 deletions(-) rename graphx/src/main/scala/org/apache/spark/graphx/util/collection/{PrimitiveKeyOpenHashMap.scala => GraphXPrimitiveKeyOpenHashMap.scala} (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index d295d0127ac72..f97f329c0e832 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -24,6 +24,9 @@ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.OpenHashSet + /** * Registers GraphX classes with Kryo for improved performance. @@ -43,8 +46,8 @@ class GraphKryoRegistrator extends KryoRegistrator { kryo.register(classOf[PartitionStrategy]) kryo.register(classOf[BoundedPriorityQueue[Object]]) kryo.register(classOf[EdgeDirection]) - - // This avoids a large number of hash table lookups. - kryo.setReferences(false) + kryo.register(classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]]) + kryo.register(classOf[OpenHashSet[Int]]) + kryo.register(classOf[OpenHashSet[Long]]) } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 871e81f8d245c..a5c9cd1f8b4e6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.{classTag, ClassTag} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A collection of edges stored in columnar format, along with any vertex attributes referenced. The @@ -42,12 +42,12 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap private[graphx] class EdgePartition[ @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag, VD: ClassTag]( - @transient val srcIds: Array[VertexId], - @transient val dstIds: Array[VertexId], - @transient val data: Array[ED], - @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int], - @transient val vertices: VertexPartition[VD], - @transient val activeSet: Option[VertexSet] = None + val srcIds: Array[VertexId] = null, + val dstIds: Array[VertexId] = null, + val data: Array[ED] = null, + val index: GraphXPrimitiveKeyOpenHashMap[VertexId, Int] = null, + val vertices: VertexPartition[VD] = null, + val activeSet: Option[VertexSet] = None ) extends Serializable { /** Return a new `EdgePartition` with the specified edge data. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index ecb49bef42e45..4520beb991515 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -23,7 +23,7 @@ import scala.util.Sorting import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: ClassTag]( @@ -41,7 +41,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla val srcIds = new Array[VertexId](edgeArray.size) val dstIds = new Array[VertexId](edgeArray.size) val data = new Array[ED](edgeArray.size) - val index = new PrimitiveKeyOpenHashMap[VertexId, Int] + val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int] // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index ebb0b9418d65d..56f79a7097fce 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -20,7 +20,7 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * The Iterator type returned when constructing edge triplets. This could be an anonymous class in diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 927e32ad0f448..d02e9238adba5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -25,7 +25,7 @@ import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that @@ -69,7 +69,7 @@ object RoutingTablePartition { : Iterator[RoutingTableMessage] = { // Determine which positions each vertex id appears in using a map where the low 2 bits // represent src and dst - val map = new PrimitiveKeyOpenHashMap[VertexId, Byte] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, Byte] edgePartition.srcIds.iterator.foreach { srcId => map.changeValue(srcId, 0x1, (b: Byte) => (b | 0x1).toByte) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index f4e221d4e05ae..dca54b8a7da86 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** Stores vertex attributes to ship to an edge partition. */ private[graphx] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index f1d174720a1ba..55c7a19d1bdab 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartition { /** Construct a `VertexPartition` from the given vertices. */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 8d9e0204d27f2..34939b24440aa 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] object VertexPartitionBase { /** @@ -32,7 +32,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)]) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map(pair._1) = pair._2 } @@ -45,7 +45,7 @@ private[graphx] object VertexPartitionBase { */ def initFrom[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD) : (VertexIdToIndexMap, Array[VD], BitSet) = { - val map = new PrimitiveKeyOpenHashMap[VertexId, VD] + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] iter.foreach { pair => map.setMerge(pair._1, pair._2, mergeFunc) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index 21ff615feca6c..a4f769b294010 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -25,7 +25,7 @@ import org.apache.spark.Logging import org.apache.spark.util.collection.BitSet import org.apache.spark.graphx._ -import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap /** * An class containing additional operations for subclasses of VertexPartitionBase that provide @@ -224,7 +224,7 @@ private[graphx] abstract class VertexPartitionBaseOps * Construct a new VertexPartition whose index contains only the vertices in the mask. */ def reindex(): Self[VD] = { - val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD] + val hashMap = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] val arbitraryMerge = (a: VD, b: VD) => a for ((k, v) <- self.iterator) { hashMap.setMerge(k, v, arbitraryMerge) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala rename to graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala index 7b02e2ed1a9cb..57b01b6f2e1fb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/collection/PrimitiveKeyOpenHashMap.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/collection/GraphXPrimitiveKeyOpenHashMap.scala @@ -29,7 +29,7 @@ import scala.reflect._ * Under the hood, it uses our OpenHashSet implementation. */ private[graphx] -class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, +class GraphXPrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag, @specialized(Long, Int, Double) V: ClassTag]( val keySet: OpenHashSet[K], var _values: Array[V]) extends Iterable[(K, V)] diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index d2e0c01bc35ef..28fd112f2b124 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -22,6 +22,9 @@ import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkConf +import org.apache.spark.serializer.KryoSerializer + import org.apache.spark.graphx._ class EdgePartitionSuite extends FunSuite { @@ -120,4 +123,19 @@ class EdgePartitionSuite extends FunSuite { assert(!ep.isActive(-1)) assert(ep.numActives == Some(2)) } + + test("Kryo serialization") { + val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) + val a: EdgePartition[Int, Int] = makeEdgePartition(aList) + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + val s = new KryoSerializer(conf).newInstance() + val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) + assert(aSer.srcIds.toList === a.srcIds.toList) + assert(aSer.dstIds.toList === a.dstIds.toList) + assert(aSer.data.toList === a.data.toList) + assert(aSer.index != null) + assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) + } } diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index fafc9b36a77d3..e147be7ddaa61 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -30,7 +30,7 @@ object MimaBuild { // Read package-private excludes from file val excludeFilePath = (base.getAbsolutePath + "/.mima-excludes") - val excludeFile = file(excludeFilePath) + val excludeFile = file(excludeFilePath) val packagePrivateList: Seq[String] = if (!excludeFile.exists()) { Seq() @@ -41,10 +41,10 @@ object MimaBuild { // Exclude a single class and its corresponding object def excludeClass(className: String) = { Seq( - excludePackage(className), + excludePackage(className), ProblemFilters.exclude[MissingClassProblem](className), ProblemFilters.exclude[MissingTypesProblem](className), - excludePackage(className + "$"), + excludePackage(className + "$"), ProblemFilters.exclude[MissingClassProblem](className + "$"), ProblemFilters.exclude[MissingTypesProblem](className + "$") ) @@ -78,6 +78,8 @@ object MimaBuild { excludeSparkClass("graphx.VertexRDD") ++ excludeSparkClass("graphx.impl.GraphImpl") ++ excludeSparkClass("graphx.impl.RoutingTable") ++ + excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") ++ + excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") ++ excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ From 1e4a65e69489ff877e6da6f78b1c1306335e373c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 13:20:23 -0700 Subject: [PATCH 23/82] BUILD: Include Hive with default packages when creating a release --- dev/create-release/create-release.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 33552a74920a7..b7a0a01c3bad2 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -109,9 +109,9 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "--hadoop 1.0.4" -make_binary_release "cdh4" "--hadoop 2.0.0-mr1-cdh4.2.0" -make_binary_release "hadoop2" "--with-yarn --hadoop 2.2.0" +make_binary_release "hadoop1" "--with-hive --hadoop 1.0.4" +make_binary_release "cdh4" "--with-hive --hadoop 2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop2" "--with-hive --with-yarn --hadoop 2.2.0" # Copy data echo "Copying release tarballs" From 7120a2979d0a9f0f54a88b2416be7ca10e74f409 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 May 2014 14:16:19 -0700 Subject: [PATCH 24/82] SPARK-1798. Tests should clean up temp files MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Three issues related to temp files that tests generate – these should be touched up for hygiene but are not urgent. Modules have a log4j.properties which directs the unit-test.log output file to a directory like `[module]/target/unit-test.log`. But this ends up creating `[module]/[module]/target/unit-test.log` instead of former. The `work/` directory is not deleted by "mvn clean", in the parent and in modules. Neither is the `checkpoint/` directory created under the various external modules. Many tests create a temp directory, which is not usually deleted. This can be largely resolved by calling `deleteOnExit()` at creation and trying to call `Utils.deleteRecursively` consistently to clean up, sometimes in an `@After` method. _If anyone seconds the motion, I can create a more significant change that introduces a new test trait along the lines of `LocalSparkContext`, which provides management of temp directories for subclasses to take advantage of._ Author: Sean Owen Closes #732 from srowen/SPARK-1798 and squashes the following commits: 5af578e [Sean Owen] Try to consistently delete test temp dirs and files, and set deleteOnExit() for each b21b356 [Sean Owen] Remove work/ and checkpoint/ dirs with mvn clean bdd0f41 [Sean Owen] Remove duplicate module dir in log4j.properties output path for tests --- bagel/src/test/resources/log4j.properties | 2 +- .../scala/org/apache/spark/TestUtils.scala | 1 + .../scala/org/apache/spark/util/Utils.scala | 18 +++--- .../java/org/apache/spark/JavaAPISuite.java | 18 ++---- core/src/test/resources/log4j.properties | 2 +- .../org/apache/spark/CheckpointSuite.scala | 5 +- .../org/apache/spark/FileServerSuite.scala | 18 ++++-- .../scala/org/apache/spark/FileSuite.scala | 56 +++++++++---------- .../WholeTextFileRecordReaderSuite.scala | 4 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 2 + .../scheduler/EventLoggingListenerSuite.scala | 15 +++-- .../spark/scheduler/ReplayListenerSuite.scala | 14 +++-- .../spark/storage/DiskBlockManagerSuite.scala | 30 +++++++--- .../apache/spark/util/FileLoggerSuite.scala | 17 ++++-- .../org/apache/spark/util/UtilsSuite.scala | 1 + .../flume/src/test/resources/log4j.properties | 2 +- .../kafka/src/test/resources/log4j.properties | 2 +- .../mqtt/src/test/resources/log4j.properties | 2 +- .../src/test/resources/log4j.properties | 2 +- .../src/test/resources/log4j.properties | 2 +- .../java/org/apache/spark/Java8APISuite.java | 3 + graphx/src/test/resources/log4j.properties | 2 +- mllib/src/test/resources/log4j.properties | 2 +- .../spark/mllib/util/MLUtilsSuite.scala | 17 ++---- pom.xml | 15 +++++ repl/src/test/resources/log4j.properties | 2 +- .../spark/repl/ExecutorClassLoaderSuite.scala | 24 ++++++-- .../org/apache/spark/repl/ReplSuite.scala | 3 + .../apache/spark/sql/InsertIntoSuite.scala | 12 ++++ .../org/apache/spark/sql/hive/TestHive.scala | 1 + .../streaming/util/MasterFailureTest.scala | 3 + .../apache/spark/streaming/JavaAPISuite.java | 5 +- streaming/src/test/resources/log4j.properties | 2 +- .../spark/streaming/CheckpointSuite.scala | 2 + .../spark/streaming/InputStreamsSuite.scala | 1 + 35 files changed, 193 insertions(+), 114 deletions(-) diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 5cdcf35b23a6c..30b4baa4d714a 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=bagel/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 8ae02154823ee..885c6829a2d72 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -43,6 +43,7 @@ private[spark] object TestUtils { */ def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = { val tempDir = Files.createTempDir() + tempDir.deleteOnExit() val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value) val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) createJar(files, jarFile) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8f7594ada2ba1..0631e54237923 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -586,15 +586,17 @@ private[spark] object Utils extends Logging { * Don't follow directories if they are symlinks. */ def deleteRecursively(file: File) { - if ((file.isDirectory) && !isSymlink(file)) { - for (child <- listFilesSafely(file)) { - deleteRecursively(child) + if (file != null) { + if ((file.isDirectory) && !isSymlink(file)) { + for (child <- listFilesSafely(file)) { + deleteRecursively(child) + } } - } - if (!file.delete()) { - // Delete can also fail if the file simply did not exist - if (file.exists()) { - throw new IOException("Failed to delete: " + file.getAbsolutePath) + if (!file.delete()) { + // Delete can also fail if the file simply did not exist + if (file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath) + } } } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 1912015827927..3dd79243ab5bd 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,7 +18,6 @@ package org.apache.spark; import java.io.*; -import java.lang.StringBuilder; import java.util.*; import scala.Tuple2; @@ -49,16 +48,20 @@ import org.apache.spark.partial.PartialResult; import org.apache.spark.storage.StorageLevel; import org.apache.spark.util.StatCounter; +import org.apache.spark.util.Utils; // 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; // see http://stackoverflow.com/questions/758570/. public class JavaAPISuite implements Serializable { private transient JavaSparkContext sc; + private transient File tempDir; @Before public void setUp() { sc = new JavaSparkContext("local", "JavaAPISuite"); + tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); } @After @@ -67,6 +70,7 @@ public void tearDown() { sc = null; // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port"); + Utils.deleteRecursively(tempDir); } static class ReverseIntComparator implements Comparator, Serializable { @@ -611,7 +615,6 @@ public void glom() { @Test public void textFiles() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir); @@ -630,7 +633,6 @@ public void wholeTextFiles() throws IOException { byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); - File tempDir = Files.createTempDir(); String tempDirName = tempDir.getAbsolutePath(); DataOutputStream ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00000")); ds.write(content1); @@ -653,7 +655,6 @@ public void wholeTextFiles() throws IOException { @Test public void textFilesCompressed() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir, DefaultCodec.class); @@ -667,7 +668,6 @@ public void textFilesCompressed() throws IOException { @SuppressWarnings("unchecked") @Test public void sequenceFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -697,7 +697,6 @@ public Tuple2 call(Tuple2 pair) { @SuppressWarnings("unchecked") @Test public void writeWithNewAPIHadoopFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -728,7 +727,6 @@ public String call(Tuple2 x) { @SuppressWarnings("unchecked") @Test public void readWithNewAPIHadoopFile() throws IOException { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -758,7 +756,6 @@ public String call(Tuple2 x) { @Test public void objectFilesOfInts() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsObjectFile(outputDir); @@ -771,7 +768,6 @@ public void objectFilesOfInts() { @SuppressWarnings("unchecked") @Test public void objectFilesOfComplexTypes() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -788,7 +784,6 @@ public void objectFilesOfComplexTypes() { @SuppressWarnings("unchecked") @Test public void hadoopFile() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -818,7 +813,6 @@ public String call(Tuple2 x) { @SuppressWarnings("unchecked") @Test public void hadoopFileCompressed() { - File tempDir = Files.createTempDir(); String outputDir = new File(tempDir, "output_compressed").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -948,7 +942,6 @@ public String call(Integer t) throws Exception { @Test public void checkpointAndComputation() { - File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); Assert.assertEquals(false, rdd.isCheckpointed()); @@ -960,7 +953,6 @@ public void checkpointAndComputation() { @Test public void checkpointAndRestore() { - File tempDir = Files.createTempDir(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); Assert.assertEquals(false, rdd.isCheckpointed()); diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index f6fef03689a7c..26b73a1b39744 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=core/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index d2555b7c052c1..64933f4b1046d 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -35,6 +35,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { override def beforeEach() { super.beforeEach() checkpointDir = File.createTempFile("temp", "") + checkpointDir.deleteOnExit() checkpointDir.delete() sc = new SparkContext("local", "test") sc.setCheckpointDir(checkpointDir.toString) @@ -42,9 +43,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { override def afterEach() { super.afterEach() - if (checkpointDir != null) { - checkpointDir.delete() - } + Utils.deleteRecursively(checkpointDir) } test("basic checkpointing") { diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index d651fbbac4e97..7e18f45de7b5b 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -24,9 +24,11 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils class FileServerSuite extends FunSuite with LocalSparkContext { + @transient var tmpDir: File = _ @transient var tmpFile: File = _ @transient var tmpJarUrl: String = _ @@ -38,15 +40,18 @@ class FileServerSuite extends FunSuite with LocalSparkContext { override def beforeAll() { super.beforeAll() - val tmpDir = new File(Files.createTempDir(), "test") - tmpDir.mkdir() - val textFile = new File(tmpDir, "FileServerSuite.txt") + tmpDir = Files.createTempDir() + tmpDir.deleteOnExit() + val testTempDir = new File(tmpDir, "test") + testTempDir.mkdir() + + val textFile = new File(testTempDir, "FileServerSuite.txt") val pw = new PrintWriter(textFile) pw.println("100") pw.close() - val jarFile = new File(tmpDir, "test.jar") + val jarFile = new File(testTempDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) System.setProperty("spark.authenticate", "false") @@ -70,6 +75,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { tmpJarUrl = jarFile.toURI.toURL.toString } + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(tmpDir) + } + test("Distributing files locally") { sc = new SparkContext("local[4]", "test") sc.addFile(tmpFile.toString) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index b9b668d3cc62a..1f2206b1f0379 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -30,12 +30,24 @@ import org.apache.hadoop.mapreduce.Job import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils class FileSuite extends FunSuite with LocalSparkContext { + var tempDir: File = _ + + override def beforeEach() { + super.beforeEach() + tempDir = Files.createTempDir() + tempDir.deleteOnExit() + } + + override def afterEach() { + super.afterEach() + Utils.deleteRecursively(tempDir) + } test("text files") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsTextFile(outputDir) @@ -49,7 +61,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("text files (compressed)") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val normalDir = new File(tempDir, "output_normal").getAbsolutePath val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath val codec = new DefaultCodec() @@ -71,7 +82,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFiles") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) // (1,a), (2,aa), (3,aaa) nums.saveAsSequenceFile(outputDir) @@ -82,7 +92,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile (compressed)") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val normalDir = new File(tempDir, "output_normal").getAbsolutePath val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath val codec = new DefaultCodec() @@ -104,7 +113,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable key") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) nums.saveAsSequenceFile(outputDir) @@ -115,7 +123,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable value") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -126,7 +133,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("SequenceFile with writable key and value") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -137,7 +143,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("implicit conversions in reading SequenceFiles") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) // (1,a), (2,aa), (3,aaa) nums.saveAsSequenceFile(outputDir) @@ -154,7 +159,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("object files of ints") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) nums.saveAsObjectFile(outputDir) @@ -165,7 +169,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("object files of complex types") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (x, "a" * x)) nums.saveAsObjectFile(outputDir) @@ -177,7 +180,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("write SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, Text]]( @@ -189,7 +191,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("read SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) @@ -200,7 +201,6 @@ class FileSuite extends FunSuite with LocalSparkContext { test("file caching") { sc = new SparkContext("local", "test") - val tempDir = Files.createTempDir() val out = new FileWriter(tempDir + "/input") out.write("Hello world!\n") out.write("What's up?\n") @@ -214,67 +214,61 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the empty directory (old Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) intercept[FileAlreadyExistsException] { - randomRDD.saveAsTextFile(tempdir.getPath) + randomRDD.saveAsTextFile(tempDir.getPath) } } test ("prevent user from overwriting the non-empty directory (old Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) - randomRDD.saveAsTextFile(tempdir.getPath + "/output") - assert(new File(tempdir.getPath + "/output/part-00000").exists() === true) + randomRDD.saveAsTextFile(tempDir.getPath + "/output") + assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) intercept[FileAlreadyExistsException] { - randomRDD.saveAsTextFile(tempdir.getPath + "/output") + randomRDD.saveAsTextFile(tempDir.getPath + "/output") } } test ("prevent user from overwriting the empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } } test ("prevent user from overwriting the non-empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath + "/output") - assert(new File(tempdir.getPath + "/output/part-r-00000").exists() === true) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } } test ("save Hadoop Dataset through old Hadoop API") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new JobConf() job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) job.set("mapred.output.format.class", classOf[TextOutputFormat[String, String]].getName) - job.set("mapred.output.dir", tempdir.getPath + "/outputDataset_old") + job.set("mapred.output.dir", tempDir.getPath + "/outputDataset_old") randomRDD.saveAsHadoopDataset(job) - assert(new File(tempdir.getPath + "/outputDataset_old/part-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_old/part-00000").exists() === true) } test ("save Hadoop Dataset through new Hadoop API") { sc = new SparkContext("local", "test") - val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new Job(sc.hadoopConfiguration) job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) job.setOutputFormatClass(classOf[NewTextOutputFormat[String, String]]) - job.getConfiguration.set("mapred.output.dir", tempdir.getPath + "/outputDataset_new") + job.getConfiguration.set("mapred.output.dir", tempDir.getPath + "/outputDataset_new") randomRDD.saveAsNewAPIHadoopDataset(job.getConfiguration) - assert(new File(tempdir.getPath + "/outputDataset_new/part-r-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true) } } diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 33d6de9a76405..d5ebfb3f3fae1 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -31,6 +31,7 @@ import org.scalatest.FunSuite import org.apache.hadoop.io.Text import org.apache.spark.SparkContext +import org.apache.spark.util.Utils /** * Tests the correctness of @@ -67,6 +68,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { test("Correctness of WholeTextFileRecordReader.") { val dir = Files.createTempDir() + dir.deleteOnExit() println(s"Local disk address is ${dir.toString}.") WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => @@ -86,7 +88,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { s"file $filename contents can not match.") } - dir.delete() + Utils.deleteRecursively(dir) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 8f3e6bd21b752..1230565ea5b7e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -236,11 +236,13 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("zero-partition RDD") { val emptyDir = Files.createTempDir() + emptyDir.deleteOnExit() val file = sc.textFile(emptyDir.getAbsolutePath) assert(file.partitions.size == 0) assert(file.collect().toList === Nil) // Test that a shuffle on the file works, because this used to be a bug assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + emptyDir.delete() } test("keys and values") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 95f5bcd855665..21e3db34b8b7a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import scala.collection.mutable import scala.io.Source -import scala.util.Try import com.google.common.io.Files import org.apache.hadoop.fs.{FileStatus, Path} @@ -30,6 +29,8 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} +import java.io.File + /** * Test whether EventLoggingListener logs events properly. * @@ -43,11 +44,17 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { "org.apache.spark.io.LZFCompressionCodec", "org.apache.spark.io.SnappyCompressionCodec" ) - private val testDir = Files.createTempDir() - private val logDirPath = Utils.getFilePath(testDir, "spark-events") + private var testDir: File = _ + private var logDirPath: Path = _ + + before { + testDir = Files.createTempDir() + testDir.deleteOnExit() + logDirPath = Utils.getFilePath(testDir, "spark-events") + } after { - Try { fileSystem.delete(logDirPath, true) } + Utils.deleteRecursively(testDir) } test("Parse names of special files") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index d1fe1fc348961..d81499ac6abef 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.scheduler -import java.io.PrintWriter - -import scala.util.Try +import java.io.{File, PrintWriter} import com.google.common.io.Files import org.json4s.jackson.JsonMethods._ @@ -39,11 +37,15 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { "org.apache.spark.io.LZFCompressionCodec", "org.apache.spark.io.SnappyCompressionCodec" ) - private val testDir = Files.createTempDir() + private var testDir: File = _ + + before { + testDir = Files.createTempDir() + testDir.deleteOnExit() + } after { - Try { fileSystem.delete(Utils.getFilePath(testDir, "events.txt"), true) } - Try { fileSystem.delete(Utils.getFilePath(testDir, "test-replay"), true) } + Utils.deleteRecursively(testDir) } test("Simple replay") { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 42bfbf1bdfc74..2167718fd2428 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -23,18 +23,16 @@ import scala.collection.mutable import scala.language.reflectiveCalls import com.google.common.io.Files -import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.SparkConf +import org.apache.spark.util.Utils -class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { +class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { private val testConf = new SparkConf(false) - val rootDir0 = Files.createTempDir() - rootDir0.deleteOnExit() - val rootDir1 = Files.createTempDir() - rootDir1.deleteOnExit() - val rootDirs = rootDir0.getName + "," + rootDir1.getName - println("Created root dirs: " + rootDirs) + private var rootDir0: File = _ + private var rootDir1: File = _ + private var rootDirs: String = _ // This suite focuses primarily on consolidation features, // so we coerce consolidation if not already enabled. @@ -48,6 +46,22 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { var diskBlockManager: DiskBlockManager = _ + override def beforeAll() { + super.beforeAll() + rootDir0 = Files.createTempDir() + rootDir0.deleteOnExit() + rootDir1 = Files.createTempDir() + rootDir1.deleteOnExit() + rootDirs = rootDir0.getName + "," + rootDir1.getName + println("Created root dirs: " + rootDirs) + } + + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(rootDir0) + Utils.deleteRecursively(rootDir1) + } + override def beforeEach() { diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) shuffleBlockManager.idToSegmentMap.clear() diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala index f675e1e5b4981..44332fc8dbc23 100644 --- a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.util -import java.io.IOException +import java.io.{File, IOException} import scala.io.Source -import scala.util.Try import com.google.common.io.Files import org.apache.hadoop.fs.Path @@ -38,12 +37,18 @@ class FileLoggerSuite extends FunSuite with BeforeAndAfter { "org.apache.spark.io.LZFCompressionCodec", "org.apache.spark.io.SnappyCompressionCodec" ) - private val testDir = Files.createTempDir() - private val logDirPath = Utils.getFilePath(testDir, "test-file-logger") - private val logDirPathString = logDirPath.toString + private var testDir: File = _ + private var logDirPath: Path = _ + private var logDirPathString: String = _ + + before { + testDir = Files.createTempDir() + logDirPath = Utils.getFilePath(testDir, "test-file-logger") + logDirPathString = logDirPath.toString + } after { - Try { fileSystem.delete(logDirPath, true) } + Utils.deleteRecursively(testDir) } test("Simple logging") { diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index eb7fb6318262b..cf9e20d347ddd 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -112,6 +112,7 @@ class UtilsSuite extends FunSuite { test("reading offset bytes of a file") { val tmpDir2 = Files.createTempDir() + tmpDir2.deleteOnExit() val f1Path = tmpDir2 + "/f1" val f1 = new FileOutputStream(f1Path) f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index d1bd73a8430e1..45d2ec676df66 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/flume/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 38910d113050a..45d2ec676df66 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/kafka/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index d0462c7336df5..45d2ec676df66 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/mqtt/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index c918335fcdc70..45d2ec676df66 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/twitter/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 304683dd0bac3..45d2ec676df66 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=external/zeromq/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index feabca6733484..84d3b6f243c72 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -39,6 +39,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; +import org.apache.spark.util.Utils; /** * Most of these tests replicate org.apache.spark.JavaAPISuite using java 8 @@ -249,6 +250,7 @@ public void mapPartitions() { @Test public void sequenceFile() { File tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); String outputDir = new File(tempDir, "output").getAbsolutePath(); List> pairs = Arrays.asList( new Tuple2(1, "a"), @@ -265,6 +267,7 @@ public void sequenceFile() { JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, Text.class) .mapToPair(pair -> new Tuple2(pair._1().get(), pair._2().toString())); Assert.assertEquals(pairs, readRDD.collect()); + Utils.deleteRecursively(tempDir); } @Test diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 85e57f0c4b504..26b73a1b39744 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=graphx/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 4265ba6e5de33..ddfc4ac6b23ed 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=mllib/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 3f64baf6fe41f..3d05fb68988c8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -32,6 +32,7 @@ import com.google.common.io.Files import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ +import org.apache.spark.util.Utils class MLUtilsSuite extends FunSuite with LocalSparkContext { @@ -67,6 +68,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { |-1 2:4.0 4:5.0 6:6.0 """.stripMargin val tempDir = Files.createTempDir() + tempDir.deleteOnExit() val file = new File(tempDir.getPath, "part-00000") Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString @@ -90,7 +92,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(multiclassPoints(1).label === -1.0) assert(multiclassPoints(2).label === -1.0) - deleteQuietly(tempDir) + Utils.deleteRecursively(tempDir) } test("saveAsLibSVMFile") { @@ -107,7 +109,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { .toSet val expected = Set("1.1 1:1.23 3:4.56", "0.0 1:1.01 2:2.02 3:3.03") assert(lines === expected) - deleteQuietly(tempDir) + Utils.deleteRecursively(tempDir) } test("appendBias") { @@ -158,16 +160,5 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } - /** Delete a file/directory quietly. */ - def deleteQuietly(f: File) { - if (f.isDirectory) { - f.listFiles().foreach(deleteQuietly) - } - try { - f.delete() - } catch { - case _: Throwable => - } - } } diff --git a/pom.xml b/pom.xml index dd1d262881b93..5542a32a91368 100644 --- a/pom.xml +++ b/pom.xml @@ -796,6 +796,21 @@ + + org.apache.maven.plugins + maven-clean-plugin + 2.5 + + + + work + + + checkpoint + + + + diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index a6d33e69d21f7..9c4896e49698c 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=repl/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index 336df988a1b7f..c0af7ceb6d3ef 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.repl import java.io.File -import java.net.URLClassLoader +import java.net.{URL, URLClassLoader} import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite @@ -26,21 +26,35 @@ import org.scalatest.FunSuite import com.google.common.io.Files import org.apache.spark.TestUtils +import org.apache.spark.util.Utils class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { val childClassNames = List("ReplFakeClass1", "ReplFakeClass2") val parentClassNames = List("ReplFakeClass1", "ReplFakeClass2", "ReplFakeClass3") - val tempDir1 = Files.createTempDir() - val tempDir2 = Files.createTempDir() - val url1 = "file://" + tempDir1 - val urls2 = List(tempDir2.toURI.toURL).toArray + var tempDir1: File = _ + var tempDir2: File = _ + var url1: String = _ + var urls2: Array[URL] = _ override def beforeAll() { + super.beforeAll() + tempDir1 = Files.createTempDir() + tempDir1.deleteOnExit() + tempDir2 = Files.createTempDir() + tempDir2.deleteOnExit() + url1 = "file://" + tempDir1 + urls2 = List(tempDir2.toURI.toURL).toArray childClassNames.foreach(TestUtils.createCompiledClass(_, tempDir1, "1")) parentClassNames.foreach(TestUtils.createCompiledClass(_, tempDir2, "2")) } + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(tempDir1) + Utils.deleteRecursively(tempDir2) + } + test("child first") { val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(url1, parentLoader, true) diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 566d96e16ed83..95460aa205331 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -26,6 +26,7 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.util.Utils class ReplSuite extends FunSuite { @@ -178,6 +179,7 @@ class ReplSuite extends FunSuite { test("interacting with files") { val tempDir = Files.createTempDir() + tempDir.deleteOnExit() val out = new FileWriter(tempDir + "/input") out.write("Hello world!\n") out.write("What's up?\n") @@ -196,6 +198,7 @@ class ReplSuite extends FunSuite { assertContains("res0: Long = 3", output) assertContains("res1: Long = 3", output) assertContains("res2: Long = 3", output) + Utils.deleteRecursively(tempDir) } test("local-cluster mode") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala index 73d87963b3a0d..4f0b85f26254b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala @@ -29,6 +29,7 @@ class InsertIntoSuite extends QueryTest { test("insertInto() created parquet file") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) testFile.registerAsTable("createAndInsertTest") @@ -76,11 +77,14 @@ class InsertIntoSuite extends QueryTest { sql("SELECT * FROM createAndInsertTest"), testData.collect().toSeq ) + + testFilePath.delete() } test("INSERT INTO parquet table") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) testFile.registerAsTable("createAndInsertSQLTest") @@ -126,23 +130,31 @@ class InsertIntoSuite extends QueryTest { sql("SELECT * FROM createAndInsertSQLTest"), testData.collect().toSeq ) + + testFilePath.delete() } test("Double create fails when allowExisting = false") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) intercept[RuntimeException] { createParquetFile[TestData](testFilePath.getCanonicalPath, allowExisting = false) } + + testFilePath.delete() } test("Double create does not fail when allowExisting = true") { val testFilePath = File.createTempFile("sparkSql", "pqt") testFilePath.delete() + testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) createParquetFile[TestData](testFilePath.getCanonicalPath, allowExisting = true) + + testFilePath.delete() } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 3ad66a3d7f45f..fa7d010459c63 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -99,6 +99,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() hiveFilesTemp.mkdir() + hiveFilesTemp.deleteOnExit() val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala index c48a38590e060..b3ed302db6a38 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala @@ -21,6 +21,7 @@ import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream.{DStream, ForEachDStream} +import org.apache.spark.util.Utils import StreamingContext._ import scala.util.Random @@ -380,6 +381,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) override def run() { val localTestDir = Files.createTempDir() + localTestDir.deleteOnExit() var fs = testDir.getFileSystem(new Configuration()) val maxTries = 3 try { @@ -421,6 +423,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) case e: Exception => logWarning("File generating in killing thread", e) } finally { fs.close() + Utils.deleteRecursively(localTestDir) } } } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index f9bfb9b7444cc..ce58cb12a4564 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,7 +17,6 @@ package org.apache.spark.streaming; -import org.apache.spark.streaming.api.java.*; import scala.Tuple2; import org.junit.Assert; @@ -37,6 +36,8 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.util.Utils; // 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; @@ -1606,6 +1607,7 @@ public void testCheckpointMasterRecovery() throws InterruptedException { Arrays.asList(8,7)); File tempDir = Files.createTempDir(); + tempDir.deleteOnExit(); ssc.checkpoint(tempDir.getAbsolutePath()); JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); @@ -1627,6 +1629,7 @@ public Integer call(String s) throws Exception { // will be re-processed after recovery List> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3); assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3)); + Utils.deleteRecursively(tempDir); } diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 063529a9cbc67..45d2ec676df66 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=streaming/target/unit-tests.log +log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 25739956cb889..d20a7b728c741 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -232,6 +232,7 @@ class CheckpointSuite extends TestSuiteBase { test("recovery with file input stream") { // Set up the streaming context and input streams val testDir = Files.createTempDir() + testDir.deleteOnExit() var ssc = new StreamingContext(master, framework, Seconds(1)) ssc.checkpoint(checkpointDir) val fileStream = ssc.textFileStream(testDir.toString) @@ -326,6 +327,7 @@ class CheckpointSuite extends TestSuiteBase { ) // To ensure that all the inputs were received correctly assert(expectedOutput.last === output.last) + Utils.deleteRecursively(testDir) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 3fa254065cc44..cd0aa4d0dce70 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -98,6 +98,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val testDir = Files.createTempDir() + testDir.deleteOnExit() val ssc = new StreamingContext(conf, batchDuration) val fileStream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] From 8586bf564fe010dfc19ef26874472a6f85e355fb Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 May 2014 14:17:25 -0700 Subject: [PATCH 25/82] SPARK-1802. Audit dependency graph when Spark is built with -Phive This initial commit resolves the conflicts in the Hive profiles as noted in https://issues.apache.org/jira/browse/SPARK-1802 . Most of the fix was to note that Hive drags in Avro, and so if the hive module depends on Spark's version of the `avro-*` dependencies, it will pull in our exclusions as needed too. But I found we need to copy some exclusions between the two Avro dependencies to get this right. And then had to squash some commons-logging intrusions. This turned up another annoying find, that `hive-exec` is basically an "assembly" artifact that _also_ packages all of its transitive dependencies. This means the final assembly shows lots of collisions between itself and its dependencies, and even other project dependencies. I have a TODO to examine whether that is going to be a deal-breaker or not. In the meantime I'm going to tack on a second commit to this PR that will also fix some similar, last collisions in the YARN profile. Author: Sean Owen Closes #744 from srowen/SPARK-1802 and squashes the following commits: a856604 [Sean Owen] Resolve JAR version conflicts specific to Hive profile --- pom.xml | 16 ++++++++++++++++ sql/hive/pom.xml | 21 +++++++++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/pom.xml b/pom.xml index 5542a32a91368..3554efafb976a 100644 --- a/pom.xml +++ b/pom.xml @@ -535,6 +535,22 @@ io.netty netty + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + org.mortbay.jetty + servlet-api + + + org.apache.velocity + velocity + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 889d249146b8c..8b32451d76045 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -52,6 +52,12 @@ org.apache.hive hive-exec ${hive.version} + + + commons-logging + commons-logging + + org.codehaus.jackson @@ -61,6 +67,21 @@ org.apache.hive hive-serde ${hive.version} + + + commons-logging + commons-logging + + + commons-logging + commons-logging-api + + + + + + org.apache.avro + avro org.scalatest From 3ce526b168050c572a1feee8e0121e1426f7d9ee Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 17:09:13 -0700 Subject: [PATCH 26/82] Rename testExecutorEnvs --> executorEnvs. This was changed, but in fact, it's used for things other than tests. So I've changed it back. Author: Patrick Wendell Closes #747 from pwendell/executor-env and squashes the following commits: 36a60a5 [Patrick Wendell] Rename testExecutorEnvs --> executorEnvs. --- .../main/scala/org/apache/spark/SparkContext.scala | 11 +++++------ .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e6121a705497c..35beff0f8ef3d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -276,19 +276,18 @@ class SparkContext(config: SparkConf) extends Logging { .getOrElse(512) // Environment variables to pass to our executors. - // NOTE: This should only be used for test related settings. - private[spark] val testExecutorEnvs = HashMap[String, String]() + private[spark] val executorEnvs = HashMap[String, String]() // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - testExecutorEnvs(envKey) = value + executorEnvs(envKey) = value } // The Mesos scheduler backend relies on this environment variable to set executor memory. // TODO: Set this only in the Mesos scheduler. - testExecutorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - testExecutorEnvs ++= conf.getExecutorEnv + executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + executorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { @@ -296,7 +295,7 @@ class SparkContext(config: SparkConf) extends Logging { }.getOrElse { SparkContext.SPARK_UNKNOWN_USER } - testExecutorEnvs("SPARK_USER") = sparkUser + executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index cefa41729964a..933f6e0571518 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -54,7 +54,7 @@ private[spark] class SparkDeploySchedulerBackend( } val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.testExecutorEnvs, + "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries, libraryPathEntries, extraJavaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 2cd9d6c12eaf7..cbe9bb093d1c9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -122,7 +122,7 @@ private[spark] class CoarseMesosSchedulerBackend( val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") - sc.testExecutorEnvs.foreach { case (key, value) => + sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 2cea1c8cf154a..ff8356f67681e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -90,7 +90,7 @@ private[spark] class MesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) val environment = Environment.newBuilder() - sc.testExecutorEnvs.foreach { case (key, value) => + sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) From 925d8b249b84d2706c52f0d1e29fb8dcd6de452e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 17:27:28 -0700 Subject: [PATCH 27/82] SPARK-1623: Use File objects instead of String's in HTTPBroadcast This seems strictly better, and I think it's justified only the grounds of clean-up. It might also fix issues with path conversions, but I haven't yet isolated any instance of that happening. /cc @srowen @tdas Author: Patrick Wendell Closes #749 from pwendell/broadcast-cleanup and squashes the following commits: d6d54f2 [Patrick Wendell] SPARK-1623: Use File objects instead of string's in HTTPBroadcast --- .../scala/org/apache/spark/broadcast/HttpBroadcast.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 78fc286e5192c..4f6cabaff2b99 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -112,7 +112,7 @@ private[spark] object HttpBroadcast extends Logging { private var securityManager: SecurityManager = null // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist - private val files = new TimeStampedHashSet[String] + private val files = new TimeStampedHashSet[File] private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt private var compressionCodec: CompressionCodec = null private var cleaner: MetadataCleaner = null @@ -173,7 +173,7 @@ private[spark] object HttpBroadcast extends Logging { val serOut = ser.serializeStream(out) serOut.writeObject(value) serOut.close() - files += file.getAbsolutePath + files += file } def read[T: ClassTag](id: Long): T = { @@ -216,7 +216,7 @@ private[spark] object HttpBroadcast extends Logging { SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) if (removeFromDriver) { val file = getFile(id) - files.remove(file.toString) + files.remove(file) deleteBroadcastFile(file) } } @@ -232,7 +232,7 @@ private[spark] object HttpBroadcast extends Logging { val (file, time) = (entry.getKey, entry.getValue) if (time < cleanupTime) { iterator.remove() - deleteBroadcastFile(new File(file.toString)) + deleteBroadcastFile(file) } } } From 4b31f4ec7efab8eabf956284a99bfd96a58b79f7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 12 May 2014 17:35:29 -0700 Subject: [PATCH 28/82] SPARK-1802. (Addendium) Audit dependency graph when Spark is built with -Pyarn Following on a few more items from SPARK-1802 -- The first commit touches up a few similar problems remaining with the YARN profile. I think this is worth cherry-picking. The second commit is more of the same for hadoop-client, although the fix is a little more complex. It may or may not be worth bothering with. Author: Sean Owen Closes #746 from srowen/SPARK-1802.2 and squashes the following commits: 52aeb41 [Sean Owen] Add more commons-logging, servlet excludes to avoid conflicts in assembly when building for YARN --- pom.xml | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/pom.xml b/pom.xml index 3554efafb976a..4d4c5f63e44a3 100644 --- a/pom.xml +++ b/pom.xml @@ -583,6 +583,10 @@ org.jboss.netty netty + + commons-logging + commons-logging + @@ -606,6 +610,10 @@ javax.servlet servlet-api + + commons-logging + commons-logging + @@ -625,6 +633,14 @@ org.jboss.netty netty + + javax.servlet + servlet-api + + + commons-logging + commons-logging + @@ -648,6 +664,10 @@ javax.servlet servlet-api + + commons-logging + commons-logging + From beb9cbaca6dbbcaba77a34df692dc2e56b3e0638 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 12 May 2014 17:39:40 -0700 Subject: [PATCH 29/82] [SPARK-1736] Spark submit for Windows Tested on Windows 7. Author: Andrew Or Closes #745 from andrewor14/windows-submit and squashes the following commits: c0b58fb [Andrew Or] Allow spaces in parameters 162e54d [Andrew Or] Merge branch 'master' of github.com:apache/spark into windows-submit 91597ce [Andrew Or] Make spark-shell.cmd use spark-submit.cmd af6fd29 [Andrew Or] Add spark submit for Windows --- bin/spark-shell.cmd | 5 ++-- bin/spark-submit.cmd | 56 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 58 insertions(+), 3 deletions(-) create mode 100644 bin/spark-submit.cmd diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 99799128eb734..ca0c722c926f5 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -17,7 +17,6 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem Find the path of sbin -set BIN=%~dp0..\bin\ +set SPARK_HOME=%~dp0.. -cmd /V /E /C %BIN%spark-class2.cmd org.apache.spark.repl.Main %* +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-internal %* --class org.apache.spark.repl.Main diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd new file mode 100644 index 0000000000000..6eb702ed8c561 --- /dev/null +++ b/bin/spark-submit.cmd @@ -0,0 +1,56 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +set SPARK_HOME=%~dp0.. +set ORIG_ARGS=%* + +rem Clear the values of all variables used +set DEPLOY_MODE= +set DRIVER_MEMORY= +set SPARK_SUBMIT_LIBRARY_PATH= +set SPARK_SUBMIT_CLASSPATH= +set SPARK_SUBMIT_OPTS= +set SPARK_DRIVER_MEMORY= + +:loop +if [%1] == [] goto continue + if [%1] == [--deploy-mode] ( + set DEPLOY_MODE=%2 + ) else if [%1] == [--driver-memory] ( + set DRIVER_MEMORY=%2 + ) else if [%1] == [--driver-library-path] ( + set SPARK_SUBMIT_LIBRARY_PATH=%2 + ) else if [%1] == [--driver-class-path] ( + set SPARK_SUBMIT_CLASSPATH=%2 + ) else if [%1] == [--driver-java-options] ( + set SPARK_SUBMIT_OPTS=%2 + ) + shift +goto loop +:continue + +if [%DEPLOY_MODE%] == [] ( + set DEPLOY_MODE=client +) + +if not [%DRIVER_MEMORY%] == [] if [%DEPLOY_MODE%] == [client] ( + set SPARK_DRIVER_MEMORY=%DRIVER_MEMORY% +) + +cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% From 3e13b8c0bd46894392bd0fbd3e86dd55b8103858 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 15:40:48 -0700 Subject: [PATCH 30/82] Adding hadoop-2.2 profile to the build --- dev/create-release/create-release.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index b7a0a01c3bad2..c4e74990860b6 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -52,14 +52,14 @@ if [[ ! "$@" =~ --package-only ]]; then -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform cd .. From 2f1a3373583f9b34a121236c25f5142ba8729546 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 12 May 2014 18:40:30 -0700 Subject: [PATCH 31/82] [SQL] Make Hive Metastore conversion functions publicly visible. I need this to be public for the implementation of SharkServer2. However, I think this functionality is generally useful and should be pretty stable. Author: Michael Armbrust Closes #750 from marmbrus/metastoreTypes and squashes the following commits: f51b62e [Michael Armbrust] Make Hive Metastore conversion functions publicly visible. --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index ba837a274c51c..fa30f596f8c67 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ @@ -172,7 +173,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with override def unregisterAllTables() = {} } -private[hive] object HiveMetastoreTypes extends RegexParsers { +/** + * :: DeveloperApi :: + * Provides conversions between Spark SQL data types and Hive Metastore types. + */ +@DeveloperApi +object HiveMetastoreTypes extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = "string" ^^^ StringType | "float" ^^^ FloatType | From a5150d199ca97ab2992bc2bb221a3ebf3d3450ba Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 12 May 2014 18:46:28 -0700 Subject: [PATCH 32/82] Typo: resond -> respond Author: Andrew Ash Closes #743 from ash211/patch-4 and squashes the following commits: c959f3b [Andrew Ash] Typo: resond -> respond --- .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index ff8356f67681e..f08b19e6782e3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -175,7 +175,7 @@ private[spark] class MesosSchedulerBackend( override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} /** - * Method called by Mesos to offer resources on slaves. We resond by asking our active task sets + * Method called by Mesos to offer resources on slaves. We respond by asking our active task sets * for tasks in order of priority. We fill each node with tasks in a round-robin manner so that * tasks are balanced across the cluster. */ From 5c2275d6e4639946fd11ff6403338c8a9ade3d1e Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Mon, 12 May 2014 19:20:24 -0700 Subject: [PATCH 33/82] L-BFGS Documentation Documentation for L-BFGS, and an example of training binary L2 logistic regression using L-BFGS. Author: DB Tsai Closes #702 from dbtsai/dbtsai-lbfgs-doc and squashes the following commits: 0712215 [DB Tsai] Update 38fdfa1 [DB Tsai] Removed extra empty line 5745b64 [DB Tsai] Update again e9e418e [DB Tsai] Update 7381521 [DB Tsai] L-BFGS Documentation --- docs/mllib-optimization.md | 120 +++++++++++++++++++++++++++++++++++-- 1 file changed, 116 insertions(+), 4 deletions(-) diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index bec3912b55dc7..aa0dec2130593 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -28,7 +28,6 @@ title: MLlib - Optimization ## Mathematical description ### Gradient descent - The simplest method to solve optimization problems of the form `$\min_{\wv \in\R^d} \; f(\wv)$` is [gradient descent](http://en.wikipedia.org/wiki/Gradient_descent). Such first-order optimization methods (including gradient descent and stochastic variants @@ -128,10 +127,19 @@ is sampled, i.e. `$|S|=$ miniBatchFraction $\cdot n = 1$`, then the algorithm is standard SGD. In that case, the step direction depends from the uniformly random sampling of the point. - +### Limited-memory BFGS (L-BFGS) +[L-BFGS](http://en.wikipedia.org/wiki/Limited-memory_BFGS) is an optimization +algorithm in the family of quasi-Newton methods to solve the optimization problems of the form +`$\min_{\wv \in\R^d} \; f(\wv)$`. The L-BFGS method approximates the objective function locally as a +quadratic without evaluating the second partial derivatives of the objective function to construct the +Hessian matrix. The Hessian matrix is approximated by previous gradient evaluations, so there is no +vertical scalability issue (the number of training features) when computing the Hessian matrix +explicitly in Newton's method. As a result, L-BFGS often achieves rapider convergence compared with +other first-order optimization. ## Implementation in MLlib +### Gradient descent and stochastic gradient descent Gradient descent methods including stochastic subgradient descent (SGD) as included as a low-level primitive in `MLlib`, upon which various ML algorithms are developed, see the @@ -142,12 +150,12 @@ The SGD method [GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) has the following parameters: -* `gradient` is a class that computes the stochastic gradient of the function +* `Gradient` is a class that computes the stochastic gradient of the function being optimized, i.e., with respect to a single training example, at the current parameter value. MLlib includes gradient classes for common loss functions, e.g., hinge, logistic, least-squares. The gradient class takes as input a training example, its label, and the current parameter value. -* `updater` is a class that performs the actual gradient descent step, i.e. +* `Updater` is a class that performs the actual gradient descent step, i.e. updating the weights in each iteration, for a given gradient of the loss part. The updater is also responsible to perform the update from the regularization part. MLlib includes updaters for cases without regularization, as well as @@ -163,3 +171,107 @@ each iteration, to compute the gradient direction. Available algorithms for gradient descent: * [GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) + +### L-BFGS +L-BFGS is currently only a low-level optimization primitive in `MLlib`. If you want to use L-BFGS in various +ML algorithms such as Linear Regression, and Logistic Regression, you have to pass the gradient of objective +function, and updater into optimizer yourself instead of using the training APIs like +[LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). +See the example below. It will be addressed in the next release. + +The L1 regularization by using +[L1Updater](api/mllib/index.html#org.apache.spark.mllib.optimization.L1Updater) will not work since the +soft-thresholding logic in L1Updater is designed for gradient descent. See the developer's note. + +The L-BFGS method +[LBFGS.runLBFGS](api/scala/index.html#org.apache.spark.mllib.optimization.LBFGS) +has the following parameters: + +* `Gradient` is a class that computes the gradient of the objective function +being optimized, i.e., with respect to a single training example, at the +current parameter value. MLlib includes gradient classes for common loss +functions, e.g., hinge, logistic, least-squares. The gradient class takes as +input a training example, its label, and the current parameter value. +* `Updater` is a class that computes the gradient and loss of objective function +of the regularization part for L-BFGS. MLlib includes updaters for cases without +regularization, as well as L2 regularizer. +* `numCorrections` is the number of corrections used in the L-BFGS update. 10 is +recommended. +* `maxNumIterations` is the maximal number of iterations that L-BFGS can be run. +* `regParam` is the regularization parameter when using regularization. + +The `return` is a tuple containing two elements. The first element is a column matrix +containing weights for every feature, and the second element is an array containing +the loss computed for every iteration. + +Here is an example to train binary logistic regression with L2 regularization using +L-BFGS optimizer. +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.classification.LogisticRegressionModel + +val data = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val numFeatures = data.take(1)(0).features.size + +// Split data into training (60%) and test (40%). +val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) + +// Append 1 into the training data as intercept. +val training = splits(0).map(x => (x.label, MLUtils.appendBias(x.features))).cache() + +val test = splits(1) + +// Run training algorithm to build the model +val numCorrections = 10 +val convergenceTol = 1e-4 +val maxNumIterations = 20 +val regParam = 0.1 +val initialWeightsWithIntercept = Vectors.dense(new Array[Double](numFeatures + 1)) + +val (weightsWithIntercept, loss) = LBFGS.runLBFGS( + training, + new LogisticGradient(), + new SquaredL2Updater(), + numCorrections, + convergenceTol, + maxNumIterations, + regParam, + initialWeightsWithIntercept) + +val model = new LogisticRegressionModel( + Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)), + weightsWithIntercept(weightsWithIntercept.size - 1)) + +// Clear the default threshold. +model.clearThreshold() + +// Compute raw scores on the test set. +val scoreAndLabels = test.map { point => + val score = model.predict(point.features) + (score, point.label) +} + +// Get evaluation metrics. +val metrics = new BinaryClassificationMetrics(scoreAndLabels) +val auROC = metrics.areaUnderROC() + +println("Loss of each step in training process") +loss.foreach(println) +println("Area under ROC = " + auROC) +{% endhighlight %} + +#### Developer's note +Since the Hessian is constructed approximately from previous gradient evaluations, +the objective function can not be changed during the optimization process. +As a result, Stochastic L-BFGS will not work naively by just using miniBatch; +therefore, we don't provide this until we have better understanding. + +* `Updater` is a class originally designed for gradient decent which computes +the actual gradient descent step. However, we're able to take the gradient and +loss of objective function of regularization for L-BFGS by ignoring the part of logic +only for gradient decent such as adaptive step size stuff. We will refactorize +this into regularizer to replace updater to separate the logic between +regularization and step update later. \ No newline at end of file From 9cf9f18973840f7287f7cfa5ce90efed3225bb30 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 12 May 2014 19:21:06 -0700 Subject: [PATCH 34/82] Modify a typo in monitoring.md As I mentioned in SPARK-1765, there is a word 'JXM' in monitoring.md. I think it's typo for 'JMX'. Author: Kousuke Saruta Closes #698 from sarutak/SPARK-1765 and squashes the following commits: bae9843 [Kousuke Saruta] modified a typoe in monitoring.md --- docs/monitoring.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/monitoring.md b/docs/monitoring.md index 6f35fc37c4075..fffc58ac39230 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -156,7 +156,7 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the * `ConsoleSink`: Logs metrics information to the console. * `CSVSink`: Exports metrics data to CSV files at regular intervals. -* `JmxSink`: Registers metrics for viewing in a JXM console. +* `JmxSink`: Registers metrics for viewing in a JMX console. * `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data. * `GraphiteSink`: Sends metrics to a Graphite node. From 156df87e7ca0e6cda2cc970ecd1466ce06f7576f Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 12 May 2014 19:23:39 -0700 Subject: [PATCH 35/82] SPARK-1757 Failing test for saving null primitives with .saveAsParquetFile() https://issues.apache.org/jira/browse/SPARK-1757 The first test succeeds, but the second test fails with exception: ``` [info] - save and load case class RDD with Nones as parquet *** FAILED *** (14 milliseconds) [info] java.lang.RuntimeException: Unsupported datatype StructType(List()) [info] at scala.sys.package$.error(package.scala:27) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$.fromDataType(ParquetRelation.scala:201) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$$anonfun$1.apply(ParquetRelation.scala:235) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$$anonfun$1.apply(ParquetRelation.scala:235) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) [info] at scala.collection.immutable.List.foreach(List.scala:318) [info] at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) [info] at scala.collection.AbstractTraversable.map(Traversable.scala:105) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$.convertFromAttributes(ParquetRelation.scala:234) [info] at org.apache.spark.sql.parquet.ParquetTypesConverter$.writeMetaData(ParquetRelation.scala:267) [info] at org.apache.spark.sql.parquet.ParquetRelation$.createEmpty(ParquetRelation.scala:143) [info] at org.apache.spark.sql.parquet.ParquetRelation$.create(ParquetRelation.scala:122) [info] at org.apache.spark.sql.execution.SparkStrategies$ParquetOperations$.apply(SparkStrategies.scala:139) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) [info] at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) [info] at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59) [info] at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan$lzycompute(SQLContext.scala:264) [info] at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan(SQLContext.scala:264) [info] at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:265) [info] at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:265) [info] at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:268) [info] at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:268) [info] at org.apache.spark.sql.SchemaRDDLike$class.saveAsParquetFile(SchemaRDDLike.scala:66) [info] at org.apache.spark.sql.SchemaRDD.saveAsParquetFile(SchemaRDD.scala:98) ``` Author: Andrew Ash Author: Michael Armbrust Closes #690 from ash211/rdd-parquet-save and squashes the following commits: 747a0b9 [Andrew Ash] Merge pull request #1 from marmbrus/pr/690 54bd00e [Michael Armbrust] Need to put Option first since Option <: Seq. 8f3f281 [Andrew Ash] SPARK-1757 Add failing test for saving SparkSQL Schemas with Option[?] fields as parquet --- .../spark/sql/catalyst/ScalaReflection.scala | 6 +-- .../spark/sql/parquet/ParquetQuerySuite.scala | 44 +++++++++++++++++++ 2 files changed, 47 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 792ef6cee6f5d..196695a0a188f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -41,6 +41,9 @@ object ScalaReflection { /** Returns a catalyst DataType for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): DataType = tpe match { + case t if t <:< typeOf[Option[_]] => + val TypeRef(_, _, Seq(optType)) = t + schemaFor(optType) case t if t <:< typeOf[Product] => val params = t.member("": TermName).asMethod.paramss StructType( @@ -59,9 +62,6 @@ object ScalaReflection { case t if t <:< typeOf[String] => StringType case t if t <:< typeOf[Timestamp] => TimestampType case t if t <:< typeOf[BigDecimal] => DecimalType - case t if t <:< typeOf[Option[_]] => - val TypeRef(_, _, Seq(optType)) = t - schemaFor(optType) case t if t <:< typeOf[java.lang.Integer] => IntegerType case t if t <:< typeOf[java.lang.Long] => LongType case t if t <:< typeOf[java.lang.Double] => DoubleType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index d9c9b9a076ab9..ff1677eb8a480 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -42,6 +42,20 @@ import org.apache.spark.sql.test.TestSQLContext._ case class TestRDDEntry(key: Int, value: String) +case class NullReflectData( + intField: java.lang.Integer, + longField: java.lang.Long, + floatField: java.lang.Float, + doubleField: java.lang.Double, + booleanField: java.lang.Boolean) + +case class OptionalReflectData( + intField: Option[Int], + longField: Option[Long], + floatField: Option[Float], + doubleField: Option[Double], + booleanField: Option[Boolean]) + class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { import TestData._ TestData // Load test data tables. @@ -195,5 +209,35 @@ class ParquetQuerySuite extends QueryTest with FunSuite with BeforeAndAfterAll { Utils.deleteRecursively(ParquetTestData.testDir) ParquetTestData.writeFile() } + + test("save and load case class RDD with nulls as parquet") { + val data = NullReflectData(null, null, null, null, null) + val rdd = sparkContext.parallelize(data :: Nil) + + val file = getTempFilePath("parquet") + val path = file.toString + rdd.saveAsParquetFile(path) + val readFile = parquetFile(path) + + val rdd_saved = readFile.collect() + assert(rdd_saved(0) === Seq.fill(5)(null)) + Utils.deleteRecursively(file) + assert(true) + } + + test("save and load case class RDD with Nones as parquet") { + val data = OptionalReflectData(null, null, null, null, null) + val rdd = sparkContext.parallelize(data :: Nil) + + val file = getTempFilePath("parquet") + val path = file.toString + rdd.saveAsParquetFile(path) + val readFile = parquetFile(path) + + val rdd_saved = readFile.collect() + assert(rdd_saved(0) === Seq.fill(5)(null)) + Utils.deleteRecursively(file) + assert(true) + } } From ba96bb3d591130075763706526f86fb2aaffa3ae Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 12 May 2014 19:42:35 -0700 Subject: [PATCH 36/82] [SPARK-1780] Non-existent SPARK_DAEMON_OPTS is lurking around What they really mean is SPARK_DAEMON_***JAVA***_OPTS Author: Andrew Or Closes #751 from andrewor14/spark-daemon-opts and squashes the following commits: 70c41f9 [Andrew Or] SPARK_DAEMON_OPTS -> SPARK_DAEMON_JAVA_OPTS --- conf/spark-env.sh.template | 2 +- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index f906be611a931..4479e1e34cd4a 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -39,5 +39,5 @@ # - SPARK_WORKER_DIR, to set the working directory of worker processes # - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") # - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") -# - SPARK_DAEMON_OPTS, to set config properties for all daemons (e.g. "-Dx=y") +# - SPARK_DAEMON_JAVA_OPTS, to set config properties for all daemons (e.g. "-Dx=y") # - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index bd21fdc5a18e4..800616622d7bf 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -247,7 +247,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application | - ./spark-submit with --driver-java-options to set -X options for a driver | - spark.executor.extraJavaOptions to set -X options for executors - | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) + | - SPARK_DAEMON_JAVA_OPTS to set java options for standalone daemons (master or worker) """.stripMargin logError(error) From 2ffd1eafd28635dcecc0ac738d4a62c05d740925 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 12 May 2014 19:44:14 -0700 Subject: [PATCH 37/82] [SPARK-1753 / 1773 / 1814] Update outdated docs for spark-submit, YARN, standalone etc. YARN - SparkPi was updated to not take in master as an argument; we should update the docs to reflect that. - The default YARN build guide should be in maven, not sbt. - This PR also adds a paragraph on steps to debug a YARN application. Standalone - Emphasize spark-submit more. Right now it's one small paragraph preceding the legacy way of launching through `org.apache.spark.deploy.Client`. - The way we set configurations / environment variables according to the old docs is outdated. This needs to reflect changes introduced by the Spark configuration changes we made. In general, this PR also adds a little more documentation on the new spark-shell, spark-submit, spark-defaults.conf etc here and there. Author: Andrew Or Closes #701 from andrewor14/yarn-docs and squashes the following commits: e2c2312 [Andrew Or] Merge in changes in #752 (SPARK-1814) 25cfe7b [Andrew Or] Merge in the warning from SPARK-1753 a8c39c5 [Andrew Or] Minor changes 336bbd9 [Andrew Or] Tabs -> spaces 4d9d8f7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 041017a [Andrew Or] Abstract Spark submit documentation to cluster-overview.html 3cc0649 [Andrew Or] Detail how to set configurations + remove legacy instructions 5b7140a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 85a51fc [Andrew Or] Update run-example, spark-shell, configuration etc. c10e8c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 381fe32 [Andrew Or] Update docs for standalone mode 757c184 [Andrew Or] Add a note about the requirements for the debugging trick f8ca990 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-docs 924f04c [Andrew Or] Revert addition of --deploy-mode d5fe17b [Andrew Or] Update the YARN docs --- conf/spark-defaults.conf.template | 3 +- conf/spark-env.sh.template | 4 +- docs/building-with-maven.md | 7 +++ docs/cluster-overview.md | 73 +++++++++++++++--------- docs/configuration.md | 64 +++++++++++++-------- docs/hadoop-third-party-distributions.md | 14 +++-- docs/index.md | 34 +++++++---- docs/java-programming-guide.md | 5 +- docs/python-programming-guide.md | 2 +- docs/quick-start.md | 4 +- docs/running-on-yarn.md | 15 +++-- docs/scala-programming-guide.md | 13 +++-- docs/spark-standalone.md | 71 ++++++++++------------- 13 files changed, 184 insertions(+), 125 deletions(-) diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template index f840ff681d019..2779342769c14 100644 --- a/conf/spark-defaults.conf.template +++ b/conf/spark-defaults.conf.template @@ -2,6 +2,7 @@ # This is useful for setting default environmental settings. # Example: -# spark.master spark://master:7077 +# spark.master spark://master:7077 # spark.eventLog.enabled true # spark.eventLog.dir hdfs://namenode:8021/directory +# spark.serializer org.apache.spark.serializer.KryoSerializer diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 4479e1e34cd4a..f8ffbf64278fb 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -30,11 +30,11 @@ # Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname -# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports +# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports for the master # - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g) -# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT +# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT, to use non-default ports for the worker # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes # - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index b6dd553bbe06b..8b44535d82404 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -129,6 +129,13 @@ Java 8 tests are run when -Pjava8-tests profile is enabled, they will run in spi For these tests to run your system must have a JDK 8 installation. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. +## Building for PySpark on YARN ## + +PySpark on YARN is only supported if the jar is built with maven. Further, there is a known problem +with building this assembly jar on Red Hat based operating systems (see SPARK-1753). If you wish to +run PySpark on a YARN cluster with Red Hat installed, we recommend that you build the jar elsewhere, +then ship it over to the cluster. We are investigating the exact cause for this. + ## Packaging without Hadoop dependencies for deployment on YARN ## The assembly jar produced by "mvn package" will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The "hadoop-provided" profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 162c415b5883e..f05a755de7fec 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -66,62 +66,76 @@ script as shown here while passing your jar. For Python, you can use the `pyFiles` argument of SparkContext or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. -### Launching Applications with ./bin/spark-submit +### Launching Applications with Spark submit Once a user application is bundled, it can be launched using the `spark-submit` script located in the bin directory. This script takes care of setting up the classpath with Spark and its -dependencies, and can support different cluster managers and deploy modes that Spark supports. -It's usage is +dependencies, and can support different cluster managers and deploy modes that Spark supports: - ./bin/spark-submit --class path.to.your.Class [options] [app options] + ./bin/spark-submit \ + --class + --master \ + --deploy-mode \ + ... // other options + + [application-arguments] -When calling `spark-submit`, `[app options]` will be passed along to your application's -main class. To enumerate all options available to `spark-submit` run it with -the `--help` flag. Here are a few examples of common options: + main-class: The entry point for your application (e.g. org.apache.spark.examples.SparkPi) + master-url: The URL of the master node (e.g. spark://23.195.26.187:7077) + deploy-mode: Whether to deploy this application within the cluster or from an external client (e.g. client) + application-jar: Path to a bundled jar including your application and all dependencies. The URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. + application-arguments: Space delimited arguments passed to the main method of , if any + +To enumerate all options available to `spark-submit` run it with the `--help` flag. Here are a few +examples of common options: {% highlight bash %} # Run application locally ./bin/spark-submit \ - --class my.main.ClassName + --class org.apache.spark.examples.SparkPi --master local[8] \ - my-app.jar + /path/to/examples.jar \ + 100 # Run on a Spark standalone cluster ./bin/spark-submit \ - --class my.main.ClassName - --master spark://mycluster:7077 \ + --class org.apache.spark.examples.SparkPi + --master spark://207.184.161.138:7077 \ --executor-memory 20G \ --total-executor-cores 100 \ - my-app.jar + /path/to/examples.jar \ + 1000 # Run on a YARN cluster -HADOOP_CONF_DIR=XX /bin/spark-submit \ - --class my.main.ClassName +HADOOP_CONF_DIR=XX ./bin/spark-submit \ + --class org.apache.spark.examples.SparkPi --master yarn-cluster \ # can also be `yarn-client` for client mode --executor-memory 20G \ --num-executors 50 \ - my-app.jar + /path/to/examples.jar \ + 1000 {% endhighlight %} ### Loading Configurations from a File -The `spark-submit` script can load default `SparkConf` values from a properties file and pass them -onto your application. By default it will read configuration options from -`conf/spark-defaults.conf`. Any values specified in the file will be passed on to the -application when run. They can obviate the need for certain flags to `spark-submit`: for -instance, if `spark.master` property is set, you can safely omit the +The `spark-submit` script can load default [Spark configuration values](configuration.html) from a +properties file and pass them on to your application. By default it will read configuration options +from `conf/spark-defaults.conf`. For more detail, see the section on +[loading default configurations](configuration.html#loading-default-configurations). + +Loading default Spark configurations this way can obviate the need for certain flags to +`spark-submit`. For instance, if the `spark.master` property is set, you can safely omit the `--master` flag from `spark-submit`. In general, configuration values explicitly set on a -`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values -in the defaults file. +`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values in the +defaults file. -If you are ever unclear where configuration options are coming from. fine-grained debugging -information can be printed by adding the `--verbose` option to `./spark-submit`. +If you are ever unclear where configuration options are coming from, you can print out fine-grained +debugging information by running `spark-submit` with the `--verbose` option. ### Advanced Dependency Management -When using `./bin/spark-submit` the app jar along with any jars included with the `--jars` option -will be automatically transferred to the cluster. `--jars` can also be used to distribute .egg and .zip -libraries for Python to executors. Spark uses the following URL scheme to allow different -strategies for disseminating jars: +When using `spark-submit`, the application jar along with any jars included with the `--jars` option +will be automatically transferred to the cluster. Spark uses the following URL scheme to allow +different strategies for disseminating jars: - **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and every executor pulls the file from the driver HTTP server. @@ -135,6 +149,9 @@ This can use up a significant amount of space over time and will need to be clea is handled automatically, and with Spark standalone, automatic cleanup can be configured with the `spark.worker.cleanup.appDataTtl` property. +For python, the equivalent `--py-files` option can be used to distribute .egg and .zip libraries +to executors. + # Monitoring Each driver program has a web UI, typically on port 4040, that displays information about running diff --git a/docs/configuration.md b/docs/configuration.md index 5b034e3cb3d47..2eed96f704a4f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -5,9 +5,9 @@ title: Spark Configuration Spark provides three locations to configure the system: -* [Spark properties](#spark-properties) control most application parameters and can be set by passing - a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java - system properties. +* [Spark properties](#spark-properties) control most application parameters and can be set by + passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, + or through the `conf/spark-defaults.conf` properties file. * [Environment variables](#environment-variables) can be used to set per-machine settings, such as the IP address, through the `conf/spark-env.sh` script on each node. * [Logging](#configuring-logging) can be configured through `log4j.properties`. @@ -15,25 +15,41 @@ Spark provides three locations to configure the system: # Spark Properties -Spark properties control most application settings and are configured separately for each application. -The preferred way to set them is by passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) -class to your SparkContext constructor. -Alternatively, Spark will also load them from Java system properties, for compatibility with old versions -of Spark. - -SparkConf lets you configure most of the common properties to initialize a cluster (e.g., master URL and -application name), as well as arbitrary key-value pairs through the `set()` method. For example, we could -initialize an application as follows: +Spark properties control most application settings and are configured separately for each +application. The preferred way is to set them through +[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) and passing it as an argument to your +SparkContext. SparkConf allows you to configure most of the common properties to initialize a +cluster (e.g. master URL and application name), as well as arbitrary key-value pairs through the +`set()` method. For example, we could initialize an application as follows: {% highlight scala %} -val conf = new SparkConf(). - setMaster("local"). - setAppName("My application"). - set("spark.executor.memory", "1g") +val conf = new SparkConf + .setMaster("local") + .setAppName("CountingSheep") + .set("spark.executor.memory", "1g") val sc = new SparkContext(conf) {% endhighlight %} -Most of the properties control internal settings that have reasonable default values. However, +## Loading Default Configurations + +In the case of `spark-shell`, a SparkContext has already been created for you, so you cannot control +the configuration properties through SparkConf. However, you can still set configuration properties +through a default configuration file. By default, `spark-shell` (and more generally `spark-submit`) +will read configuration options from `conf/spark-defaults.conf`, in which each line consists of a +key and a value separated by whitespace. For example, + + spark.master spark://5.6.7.8:7077 + spark.executor.memory 512m + spark.eventLog.enabled true + spark.serializer org.apache.spark.serializer.KryoSerializer + +Any values specified in the file will be passed on to the application, and merged with those +specified through SparkConf. If the same configuration property exists in both `spark-defaults.conf` +and SparkConf, then the latter will take precedence as it is the most application-specific. + +## All Configuration Properties + +Most of the properties that control internal settings have reasonable default values. However, there are at least five properties that you will commonly want to control: @@ -101,9 +117,9 @@ Apart from these, the following properties are also available, and may be useful @@ -696,7 +712,9 @@ Apart from these, the following properties are also available, and may be useful ## Viewing Spark Properties The application web UI at `http://:4040` lists Spark properties in the "Environment" tab. -This is a useful place to check to make sure that your properties have been set correctly. +This is a useful place to check to make sure that your properties have been set correctly. Note +that only values explicitly specified through either `spark-defaults.conf` or SparkConf will +appear. For all other configuration properties, you can assume the default value is used. # Environment Variables @@ -714,8 +732,8 @@ The following variables can be set in `spark-env.sh`: * `PYSPARK_PYTHON`, the Python binary to use for PySpark * `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. * `SPARK_PUBLIC_DNS`, the hostname your Spark program will advertise to other machines. -* Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores - to use on each machine and maximum memory. +* Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), + such as number of cores to use on each machine and maximum memory. Since `spark-env.sh` is a shell script, some of these can be set programmatically -- for example, you might compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index 454877a7fa8a5..a0aeab5727bde 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -9,12 +9,14 @@ with these distributions: # Compile-time Hadoop Version -When compiling Spark, you'll need to -[set the SPARK_HADOOP_VERSION flag](index.html#a-note-about-hadoop-versions): +When compiling Spark, you'll need to specify the Hadoop version by defining the `hadoop.version` +property. For certain versions, you will need to specify additional profiles. For more detail, +see the guide on [building with maven](building-with-maven.html#specifying-the-hadoop-version): - SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly + mvn -Dhadoop.version=1.0.4 -DskipTests clean package + mvn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package -The table below lists the corresponding `SPARK_HADOOP_VERSION` code for each CDH/HDP release. Note that +The table below lists the corresponding `hadoop.version` code for each CDH/HDP release. Note that some Hadoop releases are binary compatible across client versions. This means the pre-built Spark distribution may "just work" without you needing to compile. That said, we recommend compiling with the _exact_ Hadoop version you are running to avoid any compatibility errors. @@ -46,6 +48,10 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors.
spark.default.parallelism
    +
  • Local mode: number of cores on the local machine
  • Mesos fine grained mode: 8
  • -
  • Local mode: core number of the local machine
  • -
  • Others: total core number of all executor nodes or 2, whichever is larger
  • +
  • Others: total number of cores on all executor nodes or 2, whichever is larger
@@ -187,7 +203,7 @@ Apart from these, the following properties are also available, and may be useful Comma separated list of filter class names to apply to the Spark web ui. The filter should be a standard javax servlet Filter. Parameters to each filter can also be specified by setting a java system property of spark.<class name of filter>.params='param1=value1,param2=value2' - (e.g.-Dspark.ui.filters=com.test.filter1 -Dspark.com.test.filter1.params='param1=foo,param2=testing') + (e.g. -Dspark.ui.filters=com.test.filter1 -Dspark.com.test.filter1.params='param1=foo,param2=testing')
+In SBT, the equivalent can be achieved by setting the SPARK_HADOOP_VERSION flag: + + SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly + # Linking Applications to the Hadoop Version In addition to compiling Spark itself against the right version, you need to add a Maven dependency on that diff --git a/docs/index.md b/docs/index.md index a2f1a84371ff4..48182a27d28ae 100644 --- a/docs/index.md +++ b/docs/index.md @@ -24,21 +24,31 @@ right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/ # Running the Examples and Shell -Spark comes with several sample programs. Scala, Java and Python examples are in the `examples/src/main` directory. -To run one of the Java or Scala sample programs, use `./bin/run-example ` in the top-level Spark directory -(the `bin/run-example` script sets up the appropriate paths and launches that program). -For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`. -To run a Python sample program, use `./bin/pyspark `. For example, try `./bin/pyspark ./examples/src/main/python/pi.py local`. +Spark comes with several sample programs. Scala, Java and Python examples are in the +`examples/src/main` directory. To run one of the Java or Scala sample programs, use +`bin/run-example [params]` in the top-level Spark directory. (Behind the scenes, this +invokes the more general +[Spark submit script](cluster-overview.html#launching-applications-with-spark-submit) for +launching applications). For example, -Each example prints usage help when run with no parameters. + ./bin/run-example SparkPi 10 -Note that all of the sample programs take a `` parameter specifying the cluster URL -to connect to. This can be a [URL for a distributed cluster](scala-programming-guide.html#master-urls), -or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using -`local` for testing. +You can also run Spark interactively through modified versions of the Scala shell. This is a +great way to learn the framework. -Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or -Python interpreter (`./bin/pyspark`). These are a great way to learn the framework. + ./bin/spark-shell --master local[2] + +The `--master` option specifies the +[master URL for a distributed cluster](scala-programming-guide.html#master-urls), or `local` to run +locally with one thread, or `local[N]` to run locally with N threads. You should start by using +`local` for testing. For a full list of options, run Spark shell with the `--help` option. + +Spark also provides a Python interface. To run an example Spark application written in Python, use +`bin/pyspark [params]`. For example, + + ./bin/pyspark examples/src/main/python/pi.py local[2] 10 + +or simply `bin/pyspark` without any arguments to run Spark interactively in a python interpreter. # Launching on a Cluster diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index c34eb28fc06a2..943fdd9d019ff 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -215,7 +215,4 @@ Spark includes several sample programs using the Java API in [`examples/src/main/java`](https://github.com/apache/spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the `bin/run-example` script included in Spark; for example: - ./bin/run-example org.apache.spark.examples.JavaWordCount - -Each example program prints usage help when run -without any arguments. + ./bin/run-example JavaWordCount README.md diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 39fb5f0c99ca3..2ce2c346d7b04 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -164,6 +164,6 @@ some example applications. PySpark also includes several sample programs in the [`examples/src/main/python` folder](https://github.com/apache/spark/tree/master/examples/src/main/python). You can run them by passing the files to `pyspark`; e.g.: - ./bin/spark-submit examples/src/main/python/wordcount.py + ./bin/spark-submit examples/src/main/python/wordcount.py local[2] README.md Each program prints usage help when run without arguments. diff --git a/docs/quick-start.md b/docs/quick-start.md index 478b790f92e17..a4d01487bb494 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -18,7 +18,9 @@ you can download a package for any version of Hadoop. ## Basics Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively. -Start the shell by running `./bin/spark-shell` in the Spark directory. +Start the shell by running the following in the Spark directory. + + ./bin/spark-shell Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory: diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index c563594296802..66c330fdee734 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -54,13 +54,13 @@ For example: --executor-memory 2g \ --executor-cores 1 lib/spark-examples*.jar \ - yarn-cluster 5 + 10 The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Viewing Logs" section below for how to see driver and executor logs. To launch a Spark application in yarn-client mode, do the same, but replace "yarn-cluster" with "yarn-client". To run spark-shell: - $ MASTER=yarn-client ./bin/spark-shell + $ ./bin/spark-shell --master yarn-client ## Adding additional jars @@ -70,9 +70,9 @@ In yarn-cluster mode, the driver runs on a different machine than the client, so --master yarn-cluster \ --jars my-other-jar.jar,my-other-other-jar.jar my-main-jar.jar - yarn-cluster 5 + app_arg1 app_arg2 -# Viewing logs +# Debugging your Application In YARN terminology, executors and application masters run inside "containers". YARN has two modes for handling container logs after an application has completed. If log aggregation is turned on (with the yarn.log-aggregation-enable config), container logs are copied to HDFS and deleted on the local machine. These logs can be viewed from anywhere on the cluster with the "yarn logs" command. @@ -82,6 +82,13 @@ will print out the contents of all log files from all containers from the given When log aggregation isn't turned on, logs are retained locally on each machine under YARN_APP_LOGS_DIR, which is usually configured to /tmp/logs or $HADOOP_HOME/logs/userlogs depending on the Hadoop version and installation. Viewing logs for a container requires going to the host that contains them and looking in this directory. Subdirectories organize log files by application ID and container ID. +To review per-container launch environment, increase yarn.nodemanager.delete.debug-delay-sec to a +large value (e.g. 36000), and then access the application cache through yarn.nodemanager.local-dirs +on the nodes on which containers are launched. This directory contains the launch script, jars, and +all environment variables used for launching each container. This process is useful for debugging +classpath problems in particular. (Note that enabling this requires admin privileges on cluster +settings and a restart of all node managers. Thus, this is not applicable to hosted clusters). + # Important notes - Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index f25e9cca88524..3ed86e460c01c 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -56,7 +56,7 @@ The `master` parameter is a string specifying a [Spark, Mesos or YARN cluster UR to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. It's also possible to set these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file) -which avoids hard-coding the master name in your application. +which avoids hard-coding the master url in your application. In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the @@ -74,6 +74,11 @@ Or, to also add `code.jar` to its classpath, use: $ ./bin/spark-shell --master local[4] --jars code.jar {% endhighlight %} +For a complete list of options, run Spark shell with the `--help` option. Behind the scenes, +Spark shell invokes the more general [Spark submit script](cluster-overview.html#launching-applications-with-spark-submit) +used for launching applications, and passes on all of its parameters. As a result, these two scripts +share the same parameters. + ### Master URLs The master URL passed to Spark can be in one of the following formats: @@ -98,7 +103,7 @@ cluster mode. The cluster location will be inferred based on the local Hadoop co -If no master URL is specified, the spark shell defaults to "local[*]". +If no master URL is specified, the spark shell defaults to `local[*]`. # Resilient Distributed Datasets (RDDs) @@ -432,9 +437,7 @@ res2: Int = 10 You can see some [example Spark programs](http://spark.apache.org/examples.html) on the Spark website. In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `bin/run-example` script included in Spark; for example: - ./bin/run-example org.apache.spark.examples.SparkPi - -Each example program prints usage help when run without any arguments. + ./bin/run-example SparkPi For help on optimizing your program, the [configuration](configuration.html) and [tuning](tuning.html) guides provide information on best practices. They are especially important for diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index dc7f206e03996..eb3211b6b0e4e 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -70,7 +70,7 @@ Once you've set up this file, you can launch or stop your cluster with the follo - `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file. - `sbin/start-all.sh` - Starts both a master and a number of slaves as described above. - `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script. -- `sbin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`. +- `sbin/stop-slaves.sh` - Stops all slave instances on the machines specified in the `conf/slaves` file. - `sbin/stop-all.sh` - Stops both the master and the slaves as described above. Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine. @@ -92,12 +92,8 @@ You can optionally configure the cluster further by setting environment variable Port for the master web UI (default: 8080). - SPARK_WORKER_PORT - Start the Spark worker on a specific port (default: random). - - - SPARK_WORKER_DIR - Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work). + SPARK_MASTER_OPTS + Configuration properties that apply only to the master in the form "-Dx=y" (default: none). SPARK_WORKER_CORES @@ -107,6 +103,10 @@ You can optionally configure the cluster further by setting environment variable SPARK_WORKER_MEMORY Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GB); note that each application's individual memory is configured using its spark.executor.memory property. + + SPARK_WORKER_PORT + Start the Spark worker on a specific port (default: random). + SPARK_WORKER_WEBUI_PORT Port for the worker web UI (default: 8081). @@ -120,13 +120,25 @@ You can optionally configure the cluster further by setting environment variable or else each worker will try to use all the cores. + + SPARK_WORKER_DIR + Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work). + + + SPARK_WORKER_OPTS + Configuration properties that apply only to the worker in the form "-Dx=y" (default: none). + SPARK_DAEMON_MEMORY Memory to allocate to the Spark master and worker daemons themselves (default: 512m). SPARK_DAEMON_JAVA_OPTS - JVM options for the Spark master and worker daemons themselves (default: none). + JVM options for the Spark master and worker daemons themselves in the form "-Dx=y" (default: none). + + + SPARK_PUBLIC_DNS + The public DNS name of the Spark master and workers (default: none). @@ -148,38 +160,17 @@ You can also pass an option `--cores ` to control the number of cores # Launching Compiled Spark Applications -Spark supports two deploy modes. Spark applications may run with the driver inside the client process or entirely inside the cluster. - -The spark-submit script described in the [cluster mode overview](cluster-overview.html) provides the most straightforward way to submit a compiled Spark application to the cluster in either deploy mode. For info on the lower-level invocations used to launch an app inside the cluster, read ahead. - -## Launching Applications Inside the Cluster - - ./bin/spark-class org.apache.spark.deploy.Client launch - [client-options] \ - \ - [application-options] - - cluster-url: The URL of the master node. - application-jar-url: Path to a bundled jar including your application and all dependencies. Currently, the URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. - main-class: The entry point for your application. - - Client Options: - --memory (amount of memory, in MB, allocated for your driver program) - --cores (number of cores allocated for your driver program) - --supervise (whether to automatically restart your driver on application or node failure) - --verbose (prints increased logging output) - -Keep in mind that your driver program will be executed on a remote worker machine. You can control the execution environment in the following ways: - - * _Environment variables_: These will be captured from the environment in which you launch the client and applied when launching the driver program. - * _Java options_: You can add java options by setting `SPARK_JAVA_OPTS` in the environment in which you launch the submission client. - * _Dependencies_: You'll still need to call `sc.addJar` inside of your program to make your bundled application jar visible on all worker nodes. - -Once you submit a driver program, it will appear in the cluster management UI at port 8080 and -be assigned an identifier. If you'd like to prematurely terminate the program, you can do so using -the same client: +Spark supports two deploy modes: applications may run with the driver inside the client process or +entirely inside the cluster. The +[Spark submit script](cluster-overview.html#launching-applications-with-spark-submit) provides the +most straightforward way to submit a compiled Spark application to the cluster in either deploy +mode. - ./bin/spark-class org.apache.spark.deploy.Client kill +If your application is launched through Spark submit, then the application jar is automatically +distributed to all worker nodes. For any additional jars that your application depends on, you +should specify them through the `--jars` flag using comma as a delimiter (e.g. `--jars jar1,jar2`). +To control the application's configuration or execution environment, see +[Spark Configuration](configuration.html). # Resource Scheduling @@ -203,7 +194,7 @@ default for applications that don't set `spark.cores.max` to something less than Do this by adding the following to `conf/spark-env.sh`: {% highlight bash %} -export SPARK_JAVA_OPTS="-Dspark.deploy.defaultCores=" +export SPARK_MASTER_OPTS="-Dspark.deploy.defaultCores=" {% endhighlight %} This is useful on shared clusters where users might not have configured a maximum number of cores From 2792bd016af2a67848e6f403c4e1e05e9f3e3c2a Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 12 May 2014 20:08:30 -0700 Subject: [PATCH 38/82] SPARK-1815. SparkContext should not be marked DeveloperApi Author: Sandy Ryza Closes #753 from sryza/sandy-spark-1815 and squashes the following commits: 957a8ac [Sandy Ryza] SPARK-1815. SparkContext should not be marked DeveloperApi --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 35beff0f8ef3d..c43b4fd6d926f 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -50,7 +50,6 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** - * :: DeveloperApi :: * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * @@ -58,7 +57,6 @@ import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerTy * this config overrides the default configs as well as system properties. */ -@DeveloperApi class SparkContext(config: SparkConf) extends Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, From 52d905296fd75b3bff34a16cb69e604b68b3c246 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 May 2014 23:02:54 -0700 Subject: [PATCH 39/82] BUILD: Add more content to make-distribution.sh. --- make-distribution.sh | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/make-distribution.sh b/make-distribution.sh index 1cc2844703fbb..7a08d6b9151c4 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -171,10 +171,22 @@ echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +# Copy example sources (needed for python and SQL) +mkdir -p "$DISTDIR/examples/src/main" +cp -r $FWDIR/examples/src/main "$DISTDIR/examples/src/" + if [ "$SPARK_HIVE" == "true" ]; then cp $FWDIR/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" fi +# Copy license and ASF files +cp "$FWDIR/LICENSE" "$DISTDIR" +cp "$FWDIR/NOTICE" "$DISTDIR" + +if [ -e $FWDIR/CHANGES.txt ]; then + cp "$FWDIR/CHANGES.txt" "$DISTDIR" +fi + # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf @@ -182,6 +194,7 @@ cp "$FWDIR"/conf/slaves "$DISTDIR"/conf cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" +cp -r "$FWDIR/ec2" "$DISTDIR" # Download and copy in tachyon, if requested if [ "$SPARK_TACHYON" == "true" ]; then From 16ffadcc4af21430b5079dc555bcd9d8cf1fa1fa Mon Sep 17 00:00:00 2001 From: William Benton Date: Tue, 13 May 2014 13:45:23 -0700 Subject: [PATCH 40/82] SPARK-571: forbid return statements in cleaned closures This patch checks top-level closure arguments to `ClosureCleaner.clean` for `return` statements and raises an exception if it finds any. This is mainly a user-friendliness addition, since programs with return statements in closure arguments will currently fail upon RDD actions with a less-than-intuitive error message. Author: William Benton Closes #717 from willb/spark-571 and squashes the following commits: c41eb7d [William Benton] Another test case for SPARK-571 30c42f4 [William Benton] Stylistic cleanups 559b16b [William Benton] Stylistic cleanups from review de13b79 [William Benton] Style fixes 295b6a5 [William Benton] Forbid return statements in closure arguments. b017c47 [William Benton] Added a test for SPARK-571 --- .../apache/spark/util/ClosureCleaner.scala | 23 ++++++++++- .../spark/util/ClosureCleanerSuite.scala | 39 ++++++++++++++++++- 2 files changed, 60 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 2d05e09b10948..4916d9b86cca5 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.Set import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkException} private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -108,6 +108,9 @@ private[spark] object ClosureCleaner extends Logging { val outerObjects = getOuterObjects(func) val accessedFields = Map[Class[_], Set[String]]() + + getClassReader(func.getClass).accept(new ReturnStatementFinder(), 0) + for (cls <- outerClasses) accessedFields(cls) = Set[String]() for (cls <- func.getClass :: innerClasses) @@ -180,6 +183,24 @@ private[spark] object ClosureCleaner extends Logging { } } +private[spark] +class ReturnStatementFinder extends ClassVisitor(ASM4) { + override def visitMethod(access: Int, name: String, desc: String, + sig: String, exceptions: Array[String]): MethodVisitor = { + if (name.contains("apply")) { + new MethodVisitor(ASM4) { + override def visitTypeInsn(op: Int, tp: String) { + if (op == NEW && tp.contains("scala/runtime/NonLocalReturnControl")) { + throw new SparkException("Return statements aren't allowed in Spark closures") + } + } + } + } else { + new MethodVisitor(ASM4) {} + } + } +} + private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { override def visitMethod(access: Int, name: String, desc: String, diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index d7e48e633e0ee..054ef54e746a5 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import org.scalatest.FunSuite import org.apache.spark.LocalSparkContext._ -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} class ClosureCleanerSuite extends FunSuite { test("closures inside an object") { @@ -50,6 +50,19 @@ class ClosureCleanerSuite extends FunSuite { val obj = new TestClassWithNesting(1) assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 } + + test("toplevel return statements in closures are identified at cleaning time") { + val ex = intercept[SparkException] { + TestObjectWithBogusReturns.run() + } + + assert(ex.getMessage.contains("Return statements aren't allowed in Spark closures")) + } + + test("return statements from named functions nested in closures don't raise exceptions") { + val result = TestObjectWithNestedReturns.run() + assert(result == 1) + } } // A non-serializable class we create in closures to make sure that we aren't @@ -108,6 +121,30 @@ class TestClassWithoutFieldAccess { } } +object TestObjectWithBogusReturns { + def run(): Int = { + withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + // this return is invalid since it will transfer control outside the closure + nums.map {x => return 1 ; x * 2} + 1 + } + } +} + +object TestObjectWithNestedReturns { + def run(): Int = { + withSpark(new SparkContext("local", "test")) { sc => + val nums = sc.parallelize(Array(1, 2, 3, 4)) + nums.map {x => + // this return is fine since it will not transfer control outside the closure + def foo(): Int = { return 5; 1 } + foo() + } + 1 + } + } +} object TestObjectWithNesting { def run(): Int = { From d1e487473fd509f28daf28dcda856f3c2f1194ec Mon Sep 17 00:00:00 2001 From: Andrew Tulloch Date: Tue, 13 May 2014 17:31:27 -0700 Subject: [PATCH 41/82] SPARK-1791 - SVM implementation does not use threshold parameter Summary: https://issues.apache.org/jira/browse/SPARK-1791 Simple fix, and backward compatible, since - anyone who set the threshold was getting completely wrong answers. - anyone who did not set the threshold had the default 0.0 value for the threshold anyway. Test Plan: Unit test added that is verified to fail under the old implementation, and pass under the new implementation. Reviewers: CC: Author: Andrew Tulloch Closes #725 from ajtulloch/SPARK-1791-SVM and squashes the following commits: 770f55d [Andrew Tulloch] SPARK-1791 - SVM implementation does not use threshold parameter --- .../spark/mllib/classification/SVM.scala | 2 +- .../spark/mllib/classification/SVMSuite.scala | 37 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index e05213536e64a..316ecd713b715 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -65,7 +65,7 @@ class SVMModel private[mllib] ( intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept threshold match { - case Some(t) => if (margin < 0) 0.0 else 1.0 + case Some(t) => if (margin < t) 0.0 else 1.0 case None => margin } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index 77d6f04b32320..886c71dde3af7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -69,6 +69,43 @@ class SVMSuite extends FunSuite with LocalSparkContext { assert(numOffPredictions < input.length / 5) } + test("SVM with threshold") { + val nPoints = 10000 + + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 + val B = -1.5 + val C = 1.0 + + val testData = SVMSuite.generateSVMInput(A, Array[Double](B, C), nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val svm = new SVMWithSGD().setIntercept(true) + svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) + + val model = svm.run(testRDD) + + val validationData = SVMSuite.generateSVMInput(A, Array[Double](B, C), nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + + // Test prediction on RDD. + + var predictions = model.predict(validationRDD.map(_.features)).collect() + assert(predictions.count(_ == 0.0) != predictions.length) + + // High threshold makes all the predictions 0.0 + model.setThreshold(10000.0) + predictions = model.predict(validationRDD.map(_.features)).collect() + assert(predictions.count(_ == 0.0) == predictions.length) + + // Low threshold makes all the predictions 1.0 + model.setThreshold(-10000.0) + predictions = model.predict(validationRDD.map(_.features)).collect() + assert(predictions.count(_ == 1.0) == predictions.length) + } + test("SVM using local random SGD") { val nPoints = 10000 From 5c0dafc2c8734a421206a808b73be67b66264dd7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 13 May 2014 18:32:32 -0700 Subject: [PATCH 42/82] [SPARK-1816] LiveListenerBus dies if a listener throws an exception The solution is to wrap a try / catch / log around the posting of each event to each listener. Author: Andrew Or Closes #759 from andrewor14/listener-die and squashes the following commits: aee5107 [Andrew Or] Merge branch 'master' of github.com:apache/spark into listener-die 370939f [Andrew Or] Remove two layers of indirection 422d278 [Andrew Or] Explicitly throw an exception instead of 1 / 0 0df0e2a [Andrew Or] Try/catch and log exceptions when posting events --- .../spark/scheduler/LiveListenerBus.scala | 36 ++++++++++--- .../spark/scheduler/SparkListenerBus.scala | 50 +++++++++++++------ .../scala/org/apache/spark/util/Utils.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 50 +++++++++++++++++-- 4 files changed, 109 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index dec3316bf7745..36a6e6338faa6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.util.concurrent.{LinkedBlockingQueue, Semaphore} import org.apache.spark.Logging +import org.apache.spark.util.Utils /** * Asynchronously passes SparkListenerEvents to registered SparkListeners. @@ -42,7 +43,7 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val listenerThread = new Thread("SparkListenerBus") { setDaemon(true) - override def run() { + override def run(): Unit = Utils.logUncaughtExceptions { while (true) { eventLock.acquire() // Atomically remove and process this event @@ -77,11 +78,8 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { val eventAdded = eventQueue.offer(event) if (eventAdded) { eventLock.release() - } else if (!queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true + } else { + logQueueFullErrorMessage() } } @@ -96,13 +94,18 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { if (System.currentTimeMillis > finishTime) { return false } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ + /* Sleep rather than using wait/notify, because this is used only for testing and + * wait/notify add overhead in the general case. */ Thread.sleep(10) } true } + /** + * For testing only. Return whether the listener daemon thread is still alive. + */ + def listenerThreadIsAlive: Boolean = synchronized { listenerThread.isAlive } + /** * Return whether the event queue is empty. * @@ -111,6 +114,23 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { */ def queueIsEmpty: Boolean = synchronized { eventQueue.isEmpty } + /** + * Log an error message to indicate that the event queue is full. Do this only once. + */ + private def logQueueFullErrorMessage(): Unit = { + if (!queueFullErrorMessageLogged) { + if (listenerThread.isAlive) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with" + + "the rate at which tasks are being started by the scheduler.") + } else { + logError("SparkListenerBus thread is dead! This means SparkListenerEvents have not" + + "been (and will no longer be) propagated to listeners for some time.") + } + queueFullErrorMessageLogged = true + } + } + def stop() { if (!started) { throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 0286aac8769b2..ed9fb24bc8ce8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -20,10 +20,13 @@ package org.apache.spark.scheduler import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Logging +import org.apache.spark.util.Utils + /** * A SparkListenerEvent bus that relays events to its listeners */ -private[spark] trait SparkListenerBus { +private[spark] trait SparkListenerBus extends Logging { // SparkListeners attached to this event bus protected val sparkListeners = new ArrayBuffer[SparkListener] @@ -34,38 +37,53 @@ private[spark] trait SparkListenerBus { } /** - * Post an event to all attached listeners. This does nothing if the event is - * SparkListenerShutdown. + * Post an event to all attached listeners. + * This does nothing if the event is SparkListenerShutdown. */ def postToAll(event: SparkListenerEvent) { event match { case stageSubmitted: SparkListenerStageSubmitted => - sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) + foreachListener(_.onStageSubmitted(stageSubmitted)) case stageCompleted: SparkListenerStageCompleted => - sparkListeners.foreach(_.onStageCompleted(stageCompleted)) + foreachListener(_.onStageCompleted(stageCompleted)) case jobStart: SparkListenerJobStart => - sparkListeners.foreach(_.onJobStart(jobStart)) + foreachListener(_.onJobStart(jobStart)) case jobEnd: SparkListenerJobEnd => - sparkListeners.foreach(_.onJobEnd(jobEnd)) + foreachListener(_.onJobEnd(jobEnd)) case taskStart: SparkListenerTaskStart => - sparkListeners.foreach(_.onTaskStart(taskStart)) + foreachListener(_.onTaskStart(taskStart)) case taskGettingResult: SparkListenerTaskGettingResult => - sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) + foreachListener(_.onTaskGettingResult(taskGettingResult)) case taskEnd: SparkListenerTaskEnd => - sparkListeners.foreach(_.onTaskEnd(taskEnd)) + foreachListener(_.onTaskEnd(taskEnd)) case environmentUpdate: SparkListenerEnvironmentUpdate => - sparkListeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + foreachListener(_.onEnvironmentUpdate(environmentUpdate)) case blockManagerAdded: SparkListenerBlockManagerAdded => - sparkListeners.foreach(_.onBlockManagerAdded(blockManagerAdded)) + foreachListener(_.onBlockManagerAdded(blockManagerAdded)) case blockManagerRemoved: SparkListenerBlockManagerRemoved => - sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) + foreachListener(_.onBlockManagerRemoved(blockManagerRemoved)) case unpersistRDD: SparkListenerUnpersistRDD => - sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + foreachListener(_.onUnpersistRDD(unpersistRDD)) case applicationStart: SparkListenerApplicationStart => - sparkListeners.foreach(_.onApplicationStart(applicationStart)) + foreachListener(_.onApplicationStart(applicationStart)) case applicationEnd: SparkListenerApplicationEnd => - sparkListeners.foreach(_.onApplicationEnd(applicationEnd)) + foreachListener(_.onApplicationEnd(applicationEnd)) case SparkListenerShutdown => } } + + /** + * Apply the given function to all attached listeners, catching and logging any exception. + */ + private def foreachListener(f: SparkListener => Unit): Unit = { + sparkListeners.foreach { listener => + try { + f(listener) + } catch { + case e: Exception => + logError(s"Listener ${Utils.getFormattedClassName(listener)} threw an exception", e) + } + } + } + } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0631e54237923..99ef6dd1fa6e3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1128,7 +1128,7 @@ private[spark] object Utils extends Logging { } /** - * Executes the given block, printing and re-throwing any uncaught exceptions. + * Execute the given block, logging and re-throwing any uncaught exception. * This is particularly useful for wrapping code that runs in a thread, to ensure * that exceptions are printed, and to avoid having to catch Throwable. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 4e9fd07e68a21..5426e578a9ddd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -331,16 +331,47 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } - def checkNonZeroAvg(m: Traversable[Long], msg: String) { + test("SparkListener moves on if a listener throws an exception") { + val badListener = new BadListener + val jobCounter1 = new BasicJobCounter + val jobCounter2 = new BasicJobCounter + val bus = new LiveListenerBus + + // Propagate events to bad listener first + bus.addListener(badListener) + bus.addListener(jobCounter1) + bus.addListener(jobCounter2) + bus.start() + + // Post events to all listeners, and wait until the queue is drained + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + + // The exception should be caught, and the event should be propagated to other listeners + assert(bus.listenerThreadIsAlive) + assert(jobCounter1.count === 5) + assert(jobCounter2.count === 5) + } + + /** + * Assert that the given list of numbers has an average that is greater than zero. + */ + private def checkNonZeroAvg(m: Traversable[Long], msg: String) { assert(m.sum / m.size.toDouble > 0.0, msg) } - class BasicJobCounter extends SparkListener { + /** + * A simple listener that counts the number of jobs observed. + */ + private class BasicJobCounter extends SparkListener { var count = 0 override def onJobEnd(job: SparkListenerJobEnd) = count += 1 } - class SaveStageAndTaskInfo extends SparkListener { + /** + * A simple listener that saves all task infos and task metrics. + */ + private class SaveStageAndTaskInfo extends SparkListener { val stageInfos = mutable.Map[StageInfo, Seq[(TaskInfo, TaskMetrics)]]() var taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() @@ -358,7 +389,10 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } - class SaveTaskEvents extends SparkListener { + /** + * A simple listener that saves the task indices for all task events. + */ + private class SaveTaskEvents extends SparkListener { val startedTasks = new mutable.HashSet[Int]() val startedGettingResultTasks = new mutable.HashSet[Int]() val endedTasks = new mutable.HashSet[Int]() @@ -377,4 +411,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc startedGettingResultTasks += taskGettingResult.taskInfo.index } } + + /** + * A simple listener that throws an exception on job end. + */ + private class BadListener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd) = { throw new Exception } + } + } From 753b04dea4b04ba9d0dd0011f00e9d70367e76fc Mon Sep 17 00:00:00 2001 From: Ye Xianjin Date: Tue, 13 May 2014 19:03:51 -0700 Subject: [PATCH 43/82] [SPARK-1527] change rootDir*.getName to rootDir*.getAbsolutePath JIRA issue: [SPARK-1527](https://issues.apache.org/jira/browse/SPARK-1527) getName() only gets the last component of the file path. When deleting test-generated directories, we should pass the generated directory's absolute path to DiskBlockManager. Author: Ye Xianjin This patch had conflicts when merged, resolved by Committer: Patrick Wendell Closes #436 from advancedxy/SPARK-1527 and squashes the following commits: 4678bab [Ye Xianjin] change rootDir*.getname to rootDir*.getAbsolutePath so the temporary directories are deleted when the test is finished. --- .../scala/org/apache/spark/storage/DiskBlockManagerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 2167718fd2428..aaa7714049732 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -52,7 +52,7 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before rootDir0.deleteOnExit() rootDir1 = Files.createTempDir() rootDir1.deleteOnExit() - rootDirs = rootDir0.getName + "," + rootDir1.getName + rootDirs = rootDir0.getAbsolutePath + "," + rootDir1.getAbsolutePath println("Created root dirs: " + rootDirs) } From 44233865cf8020741d862d33cc660c88e9315dea Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 13 May 2014 21:23:51 -0700 Subject: [PATCH 44/82] [SQL] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext. Author: Michael Armbrust Closes #761 from marmbrus/existingContext and squashes the following commits: 4651051 [Michael Armbrust] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext. --- python/pyspark/sql.py | 7 +++++-- .../org/apache/spark/sql/api/java/JavaSQLContext.scala | 4 ++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 6789d7002b3b7..bbe69e7d8f89b 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -28,7 +28,7 @@ class SQLContext: register L{SchemaRDD}s as tables, execute sql over tables, cache tables, and read parquet files. """ - def __init__(self, sparkContext): + def __init__(self, sparkContext, sqlContext = None): """ Create a new SQLContext. @@ -58,10 +58,13 @@ def __init__(self, sparkContext): self._jvm = self._sc._jvm self._pythonToJavaMap = self._jvm.PythonRDD.pythonToJavaMap + if sqlContext: + self._scala_SQLContext = sqlContext + @property def _ssql_ctx(self): """ - Accessor for the JVM SparkSQL context. Subclasses can overrite this property to provide + Accessor for the JVM SparkSQL context. Subclasses can override this property to provide their own JVM Contexts. """ if not hasattr(self, '_scala_SQLContext'): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 57facbe10fc96..6f7d431b9a819 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -33,9 +33,9 @@ import org.apache.spark.util.Utils /** * The entry point for executing Spark SQL queries from a Java program. */ -class JavaSQLContext(sparkContext: JavaSparkContext) { +class JavaSQLContext(val sqlContext: SQLContext) { - val sqlContext = new SQLContext(sparkContext.sc) + def this(sparkContext: JavaSparkContext) = this(new SQLContext(sparkContext.sc)) /** * Executes a query expressed in SQL, returning the result as a JavaSchemaRDD From 92cebada09a7e5a00ab48bcb350a9462949c33eb Mon Sep 17 00:00:00 2001 From: Syed Hashmi Date: Tue, 13 May 2014 21:24:23 -0700 Subject: [PATCH 45/82] [SPARK-1784] Add a new partitioner to allow specifying # of keys per partition This change adds a new partitioner which allows users to specify # of keys per partition. Author: Syed Hashmi Closes #721 from syedhashmi/master and squashes the following commits: 4ca94cc [Syed Hashmi] [SPARK-1784] Add a new partitioner --- .../scala/org/apache/spark/Partitioner.scala | 61 +++++++++++++++++++ .../org/apache/spark/PartitioningSuite.scala | 34 +++++++++++ 2 files changed, 95 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 9155159cf6aeb..62747960618a9 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -156,3 +156,64 @@ class RangePartitioner[K : Ordering : ClassTag, V]( false } } + +/** + * A [[org.apache.spark.Partitioner]] that partitions records into specified bounds + * Default value is 1000. Once all partitions have bounds elements, the partitioner + * allocates 1 element per partition so eventually the smaller partitions are at most + * off by 1 key compared to the larger partitions. + */ +class BoundaryPartitioner[K : Ordering : ClassTag, V]( + partitions: Int, + @transient rdd: RDD[_ <: Product2[K,V]], + private val boundary: Int = 1000) + extends Partitioner { + + // this array keeps track of keys assigned to a partition + // counts[0] refers to # of keys in partition 0 and so on + private val counts: Array[Int] = { + new Array[Int](numPartitions) + } + + def numPartitions = math.abs(partitions) + + /* + * Ideally, this should've been calculated based on # partitions and total keys + * But we are not calling count on RDD here to avoid calling an action. + * User has the flexibility of calling count and passing in any appropriate boundary + */ + def keysPerPartition = boundary + + var currPartition = 0 + + /* + * Pick current partition for the key until we hit the bound for keys / partition, + * start allocating to next partition at that time. + * + * NOTE: In case where we have lets say 2000 keys and user says 3 partitions with 500 + * passed in as boundary, the first 500 will goto P1, 501-1000 go to P2, 1001-1500 go to P3, + * after that, next keys go to one partition at a time. So 1501 goes to P1, 1502 goes to P2, + * 1503 goes to P3 and so on. + */ + def getPartition(key: Any): Int = { + val partition = currPartition + counts(partition) = counts(partition) + 1 + /* + * Since we are filling up a partition before moving to next one (this helps in maintaining + * order of keys, in certain cases, it is possible to end up with empty partitions, like + * 3 partitions, 500 keys / partition and if rdd has 700 keys, 1 partition will be entirely + * empty. + */ + if(counts(currPartition) >= keysPerPartition) + currPartition = (currPartition + 1) % numPartitions + partition + } + + override def equals(other: Any): Boolean = other match { + case r: BoundaryPartitioner[_,_] => + (r.counts.sameElements(counts) && r.boundary == boundary + && r.currPartition == currPartition) + case _ => + false + } +} diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 7c30626a0c421..7d40395803f02 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -66,6 +66,40 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(descendingP4 != p4) } + test("BoundaryPartitioner equality") { + // Make an RDD where all the elements are the same so that the partition range bounds + // are deterministically all the same. + val rdd = sc.parallelize(1.to(4000)).map(x => (x, x)) + + val p2 = new BoundaryPartitioner(2, rdd, 1000) + val p4 = new BoundaryPartitioner(4, rdd, 1000) + val anotherP4 = new BoundaryPartitioner(4, rdd) + + assert(p2 === p2) + assert(p4 === p4) + assert(p2 != p4) + assert(p4 != p2) + assert(p4 === anotherP4) + assert(anotherP4 === p4) + } + + test("BoundaryPartitioner getPartition") { + val rdd = sc.parallelize(1.to(2000)).map(x => (x, x)) + val partitioner = new BoundaryPartitioner(4, rdd, 500) + 1.to(2000).map { element => { + val partition = partitioner.getPartition(element) + if (element <= 500) { + assert(partition === 0) + } else if (element > 501 && element <= 1000) { + assert(partition === 1) + } else if (element > 1001 && element <= 1500) { + assert(partition === 2) + } else if (element > 1501 && element <= 2000) { + assert(partition === 3) + } + }} + } + test("RangePartitioner getPartition") { val rdd = sc.parallelize(1.to(2000)).map(x => (x, x)) // We have different behaviour of getPartition for partitions with less than 1000 and more than From c33b8dcbf65a3a0c5ee5e65cd1dcdbc7da36aa5f Mon Sep 17 00:00:00 2001 From: larvaboy Date: Tue, 13 May 2014 21:26:08 -0700 Subject: [PATCH 46/82] Implement ApproximateCountDistinct for SparkSql Add the implementation for ApproximateCountDistinct to SparkSql. We use the HyperLogLog algorithm implemented in stream-lib, and do the count in two phases: 1) counting the number of distinct elements in each partitions, and 2) merge the HyperLogLog results from different partitions. A simple serializer and test cases are added as well. Author: larvaboy Closes #737 from larvaboy/master and squashes the following commits: bd8ef3f [larvaboy] Add support of user-provided standard deviation to ApproxCountDistinct. 9ba8360 [larvaboy] Fix alignment and null handling issues. 95b4067 [larvaboy] Add a test case for count distinct and approximate count distinct. f57917d [larvaboy] Add the parser for the approximate count. a2d5d10 [larvaboy] Add ApproximateCountDistinct aggregates and functions. 7ad273a [larvaboy] Add SparkSql serializer for HyperLogLog. 1d9aacf [larvaboy] Fix a minor typo in the toString method of the Count case class. 653542b [larvaboy] Fix a couple of minor typos. --- .../apache/spark/rdd/PairRDDFunctions.scala | 6 +- .../apache/spark/sql/catalyst/SqlParser.scala | 7 ++ .../sql/catalyst/expressions/aggregates.scala | 78 ++++++++++++++++++- .../sql/execution/SparkSqlSerializer.scala | 17 ++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 21 ++++- 5 files changed, 122 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 5efb4388f6c71..bc6d204434ad8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -217,7 +217,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Return approximate number of distinct values for each key in this RDD. * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. Uses the provided + * more accurate counts but increase the memory footprint and vice versa. Uses the provided * Partitioner to partition the output RDD. */ def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = { @@ -232,7 +232,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Return approximate number of distinct values for each key in this RDD. * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. HashPartitions the + * more accurate counts but increase the memory footprint and vice versa. HashPartitions the * output RDD into numPartitions. * */ @@ -244,7 +244,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Return approximate number of distinct values for each key this RDD. * The accuracy of approximation can be controlled through the relative standard deviation * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in - * more accurate counts but increase the memory footprint and vise versa. The default value of + * more accurate counts but increase the memory footprint and vice versa. The default value of * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism * level. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index b3a3a1ef1b5eb..f2b9b2c1a3ad5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -93,6 +93,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val AND = Keyword("AND") protected val AS = Keyword("AS") protected val ASC = Keyword("ASC") + protected val APPROXIMATE = Keyword("APPROXIMATE") protected val AVG = Keyword("AVG") protected val BY = Keyword("BY") protected val CAST = Keyword("CAST") @@ -318,6 +319,12 @@ class SqlParser extends StandardTokenParsers with PackratParsers { COUNT ~> "(" ~ "*" <~ ")" ^^ { case _ => Count(Literal(1)) } | COUNT ~> "(" ~ expression <~ ")" ^^ { case dist ~ exp => Count(exp) } | COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => CountDistinct(exp :: Nil) } | + APPROXIMATE ~> COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { + case exp => ApproxCountDistinct(exp) + } | + APPROXIMATE ~> "(" ~> floatLit ~ ")" ~ COUNT ~ "(" ~ DISTINCT ~ expression <~ ")" ^^ { + case s ~ _ ~ _ ~ _ ~ _ ~ e => ApproxCountDistinct(e, s.toDouble) + } | FIRST ~> "(" ~> expression <~ ")" ^^ { case exp => First(exp) } | AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } | MIN ~> "(" ~> expression <~ ")" ^^ { case exp => Min(exp) } | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 7777d372903e2..5dbaaa3b0ce35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import com.clearspring.analytics.stream.cardinality.HyperLogLog + import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException @@ -146,7 +148,6 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr override def eval(input: Row): Any = currentMax } - case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references override def nullable = false @@ -166,10 +167,47 @@ case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpressi override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType - override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")}})" + override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" override def newInstance() = new CountDistinctFunction(expressions, this) } +case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) + extends AggregateExpression with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = false + override def dataType = child.dataType + override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" + override def newInstance() = new ApproxCountDistinctPartitionFunction(child, this, relativeSD) +} + +case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) + extends AggregateExpression with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = false + override def dataType = IntegerType + override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" + override def newInstance() = new ApproxCountDistinctMergeFunction(child, this, relativeSD) +} + +case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) + extends PartialAggregate with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = false + override def dataType = IntegerType + override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" + + override def asPartial: SplitEvaluation = { + val partialCount = + Alias(ApproxCountDistinctPartition(child, relativeSD), "PartialApproxCountDistinct")() + + SplitEvaluation( + ApproxCountDistinctMerge(partialCount.toAttribute, relativeSD), + partialCount :: Nil) + } + + override def newInstance() = new CountDistinctFunction(child :: Nil, this) +} + case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def references = child.references override def nullable = false @@ -269,6 +307,42 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag override def eval(input: Row): Any = count } +case class ApproxCountDistinctPartitionFunction( + expr: Expression, + base: AggregateExpression, + relativeSD: Double) + extends AggregateFunction { + def this() = this(null, null, 0) // Required for serialization. + + private val hyperLogLog = new HyperLogLog(relativeSD) + + override def update(input: Row): Unit = { + val evaluatedExpr = expr.eval(input) + if (evaluatedExpr != null) { + hyperLogLog.offer(evaluatedExpr) + } + } + + override def eval(input: Row): Any = hyperLogLog +} + +case class ApproxCountDistinctMergeFunction( + expr: Expression, + base: AggregateExpression, + relativeSD: Double) + extends AggregateFunction { + def this() = this(null, null, 0) // Required for serialization. + + private val hyperLogLog = new HyperLogLog(relativeSD) + + override def update(input: Row): Unit = { + val evaluatedExpr = expr.eval(input) + hyperLogLog.addAll(evaluatedExpr.asInstanceOf[HyperLogLog]) + } + + override def eval(input: Row): Any = hyperLogLog.cardinality() +} + case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 1c6e29b3cdee9..94c2a249ef8f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import scala.reflect.ClassTag +import com.clearspring.analytics.stream.cardinality.HyperLogLog import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} @@ -44,6 +45,8 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[scala.collection.Map[_,_]], new MapSerializer) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow]) + kryo.register(classOf[com.clearspring.analytics.stream.cardinality.HyperLogLog], + new HyperLogLogSerializer) kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]]) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) kryo.setReferences(false) @@ -81,6 +84,20 @@ private[sql] class BigDecimalSerializer extends Serializer[BigDecimal] { } } +private[sql] class HyperLogLogSerializer extends Serializer[HyperLogLog] { + def write(kryo: Kryo, output: Output, hyperLogLog: HyperLogLog) { + val bytes = hyperLogLog.getBytes() + output.writeInt(bytes.length) + output.writeBytes(bytes) + } + + def read(kryo: Kryo, input: Input, tpe: Class[HyperLogLog]): HyperLogLog = { + val length = input.readInt() + val bytes = input.readBytes(length) + HyperLogLog.Builder.build(bytes) + } +} + /** * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize * them as `Array[(k,v)]`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index e966d89c30cf5..524549eb544fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -96,8 +96,25 @@ class SQLQuerySuite extends QueryTest { test("count") { checkAnswer( sql("SELECT COUNT(*) FROM testData2"), - testData2.count() - ) + testData2.count()) + } + + test("count distinct") { + checkAnswer( + sql("SELECT COUNT(DISTINCT b) FROM testData2"), + 2) + } + + test("approximate count distinct") { + checkAnswer( + sql("SELECT APPROXIMATE COUNT(DISTINCT a) FROM testData2"), + 3) + } + + test("approximate count distinct with user provided standard deviation") { + checkAnswer( + sql("SELECT APPROXIMATE(0.04) COUNT(DISTINCT a) FROM testData2"), + 3) } // No support for primitive nulls yet. From 7bb9a521f35eb19576c6cc2da3fd385910270e46 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 13 May 2014 23:24:51 -0700 Subject: [PATCH 47/82] Revert "[SPARK-1784] Add a new partitioner to allow specifying # of keys per partition" This reverts commit 92cebada09a7e5a00ab48bcb350a9462949c33eb. --- .../scala/org/apache/spark/Partitioner.scala | 61 ------------------- .../org/apache/spark/PartitioningSuite.scala | 34 ----------- 2 files changed, 95 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 62747960618a9..9155159cf6aeb 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -156,64 +156,3 @@ class RangePartitioner[K : Ordering : ClassTag, V]( false } } - -/** - * A [[org.apache.spark.Partitioner]] that partitions records into specified bounds - * Default value is 1000. Once all partitions have bounds elements, the partitioner - * allocates 1 element per partition so eventually the smaller partitions are at most - * off by 1 key compared to the larger partitions. - */ -class BoundaryPartitioner[K : Ordering : ClassTag, V]( - partitions: Int, - @transient rdd: RDD[_ <: Product2[K,V]], - private val boundary: Int = 1000) - extends Partitioner { - - // this array keeps track of keys assigned to a partition - // counts[0] refers to # of keys in partition 0 and so on - private val counts: Array[Int] = { - new Array[Int](numPartitions) - } - - def numPartitions = math.abs(partitions) - - /* - * Ideally, this should've been calculated based on # partitions and total keys - * But we are not calling count on RDD here to avoid calling an action. - * User has the flexibility of calling count and passing in any appropriate boundary - */ - def keysPerPartition = boundary - - var currPartition = 0 - - /* - * Pick current partition for the key until we hit the bound for keys / partition, - * start allocating to next partition at that time. - * - * NOTE: In case where we have lets say 2000 keys and user says 3 partitions with 500 - * passed in as boundary, the first 500 will goto P1, 501-1000 go to P2, 1001-1500 go to P3, - * after that, next keys go to one partition at a time. So 1501 goes to P1, 1502 goes to P2, - * 1503 goes to P3 and so on. - */ - def getPartition(key: Any): Int = { - val partition = currPartition - counts(partition) = counts(partition) + 1 - /* - * Since we are filling up a partition before moving to next one (this helps in maintaining - * order of keys, in certain cases, it is possible to end up with empty partitions, like - * 3 partitions, 500 keys / partition and if rdd has 700 keys, 1 partition will be entirely - * empty. - */ - if(counts(currPartition) >= keysPerPartition) - currPartition = (currPartition + 1) % numPartitions - partition - } - - override def equals(other: Any): Boolean = other match { - case r: BoundaryPartitioner[_,_] => - (r.counts.sameElements(counts) && r.boundary == boundary - && r.currPartition == currPartition) - case _ => - false - } -} diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 7d40395803f02..7c30626a0c421 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -66,40 +66,6 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(descendingP4 != p4) } - test("BoundaryPartitioner equality") { - // Make an RDD where all the elements are the same so that the partition range bounds - // are deterministically all the same. - val rdd = sc.parallelize(1.to(4000)).map(x => (x, x)) - - val p2 = new BoundaryPartitioner(2, rdd, 1000) - val p4 = new BoundaryPartitioner(4, rdd, 1000) - val anotherP4 = new BoundaryPartitioner(4, rdd) - - assert(p2 === p2) - assert(p4 === p4) - assert(p2 != p4) - assert(p4 != p2) - assert(p4 === anotherP4) - assert(anotherP4 === p4) - } - - test("BoundaryPartitioner getPartition") { - val rdd = sc.parallelize(1.to(2000)).map(x => (x, x)) - val partitioner = new BoundaryPartitioner(4, rdd, 500) - 1.to(2000).map { element => { - val partition = partitioner.getPartition(element) - if (element <= 500) { - assert(partition === 0) - } else if (element > 501 && element <= 1000) { - assert(partition === 1) - } else if (element > 1001 && element <= 1500) { - assert(partition === 2) - } else if (element > 1501 && element <= 2000) { - assert(partition === 3) - } - }} - } - test("RangePartitioner getPartition") { val rdd = sc.parallelize(1.to(2000)).map(x => (x, x)) // We have different behaviour of getPartition for partitions with less than 1000 and more than From 6ce0884446d3571fd6e9d967a080a59c657543b1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 13 May 2014 23:27:22 -0700 Subject: [PATCH 48/82] [SQL] Improve column pruning. Fixed a bug that was preventing us from ever pruning beneath Joins. ## TPC-DS Q3 ### Before: ``` Aggregate false, [d_year#12,i_brand#65,i_brand_id#64], [d_year#12,i_brand_id#64 AS brand_id#0,i_brand#65 AS brand#1,SUM(PartialSum#79) AS sum_agg#2] Exchange (HashPartitioning [d_year#12:0,i_brand#65:1,i_brand_id#64:2], 150) Aggregate true, [d_year#12,i_brand#65,i_brand_id#64], [d_year#12,i_brand#65,i_brand_id#64,SUM(CAST(ss_ext_sales_price#49, DoubleType)) AS PartialSum#79] Project [d_year#12:6,i_brand#65:59,i_brand_id#64:58,ss_ext_sales_price#49:43] HashJoin [ss_item_sk#36], [i_item_sk#57], BuildRight Exchange (HashPartitioning [ss_item_sk#36:30], 150) HashJoin [d_date_sk#6], [ss_sold_date_sk#34], BuildRight Exchange (HashPartitioning [d_date_sk#6:0], 150) Filter (d_moy#14:8 = 12) HiveTableScan [d_date_sk#6,d_date_id#7,d_date#8,d_month_seq#9,d_week_seq#10,d_quarter_seq#11,d_year#12,d_dow#13,d_moy#14,d_dom#15,d_qoy#16,d_fy_year#17,d_fy_quarter_seq#18,d_fy_week_seq#19,d_day_name#20,d_quarter_name#21,d_holiday#22,d_weekend#23,d_following_holiday#24,d_first_dom#25,d_last_dom#26,d_same_day_ly#27,d_same_day_lq#28,d_current_day#29,d_current_week#30,d_current_month#31,d_current_quarter#32,d_current_year#33], (MetastoreRelation default, date_dim, Some(dt)), None Exchange (HashPartitioning [ss_sold_date_sk#34:0], 150) HiveTableScan [ss_sold_date_sk#34,ss_sold_time_sk#35,ss_item_sk#36,ss_customer_sk#37,ss_cdemo_sk#38,ss_hdemo_sk#39,ss_addr_sk#40,ss_store_sk#41,ss_promo_sk#42,ss_ticket_number#43,ss_quantity#44,ss_wholesale_cost#45,ss_list_price#46,ss_sales_price#47,ss_ext_discount_amt#48,ss_ext_sales_price#49,ss_ext_wholesale_cost#50,ss_ext_list_price#51,ss_ext_tax#52,ss_coupon_amt#53,ss_net_paid#54,ss_net_paid_inc_tax#55,ss_net_profit#56], (MetastoreRelation default, store_sales, None), None Exchange (HashPartitioning [i_item_sk#57:0], 150) Filter (i_manufact_id#70:13 = 436) HiveTableScan [i_item_sk#57,i_item_id#58,i_rec_start_date#59,i_rec_end_date#60,i_item_desc#61,i_current_price#62,i_wholesale_cost#63,i_brand_id#64,i_brand#65,i_class_id#66,i_class#67,i_category_id#68,i_category#69,i_manufact_id#70,i_manufact#71,i_size#72,i_formulation#73,i_color#74,i_units#75,i_container#76,i_manager_id#77,i_product_name#78], (MetastoreRelation default, item, None), None ``` ### After ``` Aggregate false, [d_year#172,i_brand#225,i_brand_id#224], [d_year#172,i_brand_id#224 AS brand_id#160,i_brand#225 AS brand#161,SUM(PartialSum#239) AS sum_agg#162] Exchange (HashPartitioning [d_year#172:0,i_brand#225:1,i_brand_id#224:2], 150) Aggregate true, [d_year#172,i_brand#225,i_brand_id#224], [d_year#172,i_brand#225,i_brand_id#224,SUM(CAST(ss_ext_sales_price#209, DoubleType)) AS PartialSum#239] Project [d_year#172:1,i_brand#225:5,i_brand_id#224:3,ss_ext_sales_price#209:0] HashJoin [ss_item_sk#196], [i_item_sk#217], BuildRight Exchange (HashPartitioning [ss_item_sk#196:2], 150) Project [ss_ext_sales_price#209:2,d_year#172:1,ss_item_sk#196:3] HashJoin [d_date_sk#166], [ss_sold_date_sk#194], BuildRight Exchange (HashPartitioning [d_date_sk#166:0], 150) Project [d_date_sk#166:0,d_year#172:1] Filter (d_moy#174:2 = 12) HiveTableScan [d_date_sk#166,d_year#172,d_moy#174], (MetastoreRelation default, date_dim, Some(dt)), None Exchange (HashPartitioning [ss_sold_date_sk#194:2], 150) HiveTableScan [ss_ext_sales_price#209,ss_item_sk#196,ss_sold_date_sk#194], (MetastoreRelation default, store_sales, None), None Exchange (HashPartitioning [i_item_sk#217:1], 150) Project [i_brand_id#224:0,i_item_sk#217:1,i_brand#225:2] Filter (i_manufact_id#230:3 = 436) HiveTableScan [i_brand_id#224,i_item_sk#217,i_brand#225,i_manufact_id#230], (MetastoreRelation default, item, None), None ``` Author: Michael Armbrust Closes #729 from marmbrus/fixPruning and squashes the following commits: 5feeff0 [Michael Armbrust] Improve column pruning. --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 3037d45cc6e35..406ffd6801e98 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -25,13 +25,13 @@ import org.apache.spark.sql.catalyst.types._ object Optimizer extends RuleExecutor[LogicalPlan] { val batches = - Batch("ConstantFolding", Once, + Batch("ConstantFolding", FixedPoint(100), NullPropagation, ConstantFolding, BooleanSimplification, SimplifyFilters, SimplifyCasts) :: - Batch("Filter Pushdown", Once, + Batch("Filter Pushdown", FixedPoint(100), CombineFilters, PushPredicateThroughProject, PushPredicateThroughInnerJoin, @@ -49,17 +49,19 @@ object Optimizer extends RuleExecutor[LogicalPlan] { */ object ColumnPruning extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Eliminate attributes that are not needed to calculate the specified aggregates. case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty => - // Project away references that are not needed to calculate the required aggregates. a.copy(child = Project(a.references.toSeq, child)) + // Eliminate unneeded attributes from either side of a Join. case Project(projectList, Join(left, right, joinType, condition)) => // Collect the list of off references required either above or to evaluate the condition. val allReferences: Set[Attribute] = projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) - /** Applies a projection when the child is producing unnecessary attributes */ + + /** Applies a projection only when the child is producing unnecessary attributes */ def prunedChild(c: LogicalPlan) = - if ((allReferences.filter(c.outputSet.contains) -- c.outputSet).nonEmpty) { + if ((c.outputSet -- allReferences.filter(c.outputSet.contains)).nonEmpty) { Project(allReferences.filter(c.outputSet.contains).toSeq, c) } else { c @@ -67,6 +69,7 @@ object ColumnPruning extends Rule[LogicalPlan] { Project(projectList, Join(prunedChild(left), prunedChild(right), joinType, condition)) + // Combine adjacent Projects. case Project(projectList1, Project(projectList2, child)) => // Create a map of Aliases to their values from the child projection. // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). @@ -83,6 +86,9 @@ object ColumnPruning extends Rule[LogicalPlan] { }).asInstanceOf[Seq[NamedExpression]] Project(substitutedProjection, child) + + // Eliminate no-op Projects + case Project(projectList, child) if(child.output == projectList) => child } } From b22952fa1f21c0b93208846b5e1941a9d2578c6f Mon Sep 17 00:00:00 2001 From: Koert Kuipers Date: Wed, 14 May 2014 00:10:12 -0700 Subject: [PATCH 49/82] SPARK-1801. expose InterruptibleIterator and TaskKilledException in deve... ...loper api Author: Koert Kuipers Closes #764 from koertkuipers/feat-rdd-developerapi and squashes the following commits: 8516dd2 [Koert Kuipers] SPARK-1801. expose InterruptibleIterator and TaskKilledException in developer api --- .../scala/org/apache/spark/InterruptibleIterator.scala | 6 +++++- .../main/scala/org/apache/spark/TaskKilledException.scala | 8 ++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index ec11dbbffaaf8..f40baa8e43592 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -17,11 +17,15 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * An iterator that wraps around an existing iterator to provide task killing functionality. * It works by checking the interrupted flag in [[TaskContext]]. */ -private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) +@DeveloperApi +class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { def hasNext: Boolean = { diff --git a/core/src/main/scala/org/apache/spark/TaskKilledException.scala b/core/src/main/scala/org/apache/spark/TaskKilledException.scala index cbd6b2866e4f9..ad487c4efb87a 100644 --- a/core/src/main/scala/org/apache/spark/TaskKilledException.scala +++ b/core/src/main/scala/org/apache/spark/TaskKilledException.scala @@ -17,7 +17,11 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi + /** - * Exception for a task getting killed. + * :: DeveloperApi :: + * Exception thrown when a task is explicitly killed (i.e., task failure is expected). */ -private[spark] class TaskKilledException extends RuntimeException +@DeveloperApi +class TaskKilledException extends RuntimeException From 54ae8328bd7d052ba347768cfb02cb5dfdd8045e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 May 2014 00:37:57 -0700 Subject: [PATCH 50/82] Fix dep exclusion: avro-ipc, not avro, depends on netty. Author: Marcelo Vanzin Closes #763 from vanzin/netty-dep-hell and squashes the following commits: dfb6ce2 [Marcelo Vanzin] Fix dep exclusion: avro-ipc, not avro, depends on netty. --- pom.xml | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 4d4c5f63e44a3..786b6d4984d86 100644 --- a/pom.xml +++ b/pom.xml @@ -496,18 +496,16 @@ org.apache.avro avro ${avro.version} - - - io.netty - netty - -
org.apache.avro avro-ipc ${avro.version} + + io.netty + netty + org.mortbay.jetty jetty From 69f750228f3ec8537a93da08e712596fa8004143 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 14 May 2014 00:54:33 -0700 Subject: [PATCH 51/82] [SPARK-1769] Executor loss causes NPE race condition This PR replaces the Schedulable data structures in Pool.scala with thread-safe ones from java. Note that Scala's `with SynchronizedBuffer` trait is soon to be deprecated in 2.11 because it is ["inherently unreliable"](http://www.scala-lang.org/api/2.11.0/index.html#scala.collection.mutable.SynchronizedBuffer). We should slowly drift away from `SynchronizedBuffer` in other places too. Note that this PR introduces an API-breaking change; `sc.getAllPools` now returns an Array rather than an ArrayBuffer. This is because we want this method to return an immutable copy rather than one may potentially confuse the user if they try to modify the copy, which takes no effect on the original data structure. Author: Andrew Or Closes #762 from andrewor14/pool-npe and squashes the following commits: 383e739 [Andrew Or] JavaConverters -> JavaConversions 3f32981 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pool-npe 769be19 [Andrew Or] Assorted minor changes 2189247 [Andrew Or] Merge branch 'master' of github.com:apache/spark into pool-npe 05ad9e9 [Andrew Or] Fix test - contains is not the same as containsKey 0921ea0 [Andrew Or] var -> val 07d720c [Andrew Or] Synchronize Schedulable data structures --- .../scala/org/apache/spark/SparkContext.scala | 20 +++++++----- .../org/apache/spark/scheduler/Pool.scala | 31 ++++++++++--------- .../apache/spark/scheduler/Schedulable.scala | 6 ++-- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../scheduler/TaskSchedulerImplSuite.scala | 2 +- 5 files changed, 35 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c43b4fd6d926f..032b3d744c619 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -17,15 +17,17 @@ package org.apache.spark +import scala.language.implicitConversions + import java.io._ import java.net.URI import java.util.concurrent.atomic.AtomicInteger import java.util.{Properties, UUID} import java.util.UUID.randomUUID import scala.collection.{Map, Set} +import scala.collection.JavaConversions._ import scala.collection.generic.Growable -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.language.implicitConversions +import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -836,18 +838,22 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Return pools for fair scheduler - * TODO(xiajunluan): We should take nested pools into account + * :: DeveloperApi :: + * Return pools for fair scheduler */ - def getAllPools: ArrayBuffer[Schedulable] = { - taskScheduler.rootPool.schedulableQueue + @DeveloperApi + def getAllPools: Seq[Schedulable] = { + // TODO(xiajunluan): We should take nested pools into account + taskScheduler.rootPool.schedulableQueue.toSeq } /** + * :: DeveloperApi :: * Return the pool associated with the given name, if one exists */ + @DeveloperApi def getPoolForName(pool: String): Option[Schedulable] = { - taskScheduler.rootPool.schedulableNameToSchedulable.get(pool) + Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool)) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 187672c4e19e7..174b73221afc0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -17,8 +17,10 @@ package org.apache.spark.scheduler +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} + +import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap import org.apache.spark.Logging import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -35,18 +37,15 @@ private[spark] class Pool( extends Schedulable with Logging { - var schedulableQueue = new ArrayBuffer[Schedulable] - var schedulableNameToSchedulable = new HashMap[String, Schedulable] - + val schedulableQueue = new ConcurrentLinkedQueue[Schedulable] + val schedulableNameToSchedulable = new ConcurrentHashMap[String, Schedulable] var weight = initWeight var minShare = initMinShare var runningTasks = 0 - var priority = 0 // A pool's stage id is used to break the tie in scheduling. var stageId = -1 - var name = poolName var parent: Pool = null @@ -60,19 +59,20 @@ private[spark] class Pool( } override def addSchedulable(schedulable: Schedulable) { - schedulableQueue += schedulable - schedulableNameToSchedulable(schedulable.name) = schedulable + require(schedulable != null) + schedulableQueue.add(schedulable) + schedulableNameToSchedulable.put(schedulable.name, schedulable) schedulable.parent = this } override def removeSchedulable(schedulable: Schedulable) { - schedulableQueue -= schedulable - schedulableNameToSchedulable -= schedulable.name + schedulableQueue.remove(schedulable) + schedulableNameToSchedulable.remove(schedulable.name) } override def getSchedulableByName(schedulableName: String): Schedulable = { - if (schedulableNameToSchedulable.contains(schedulableName)) { - return schedulableNameToSchedulable(schedulableName) + if (schedulableNameToSchedulable.containsKey(schedulableName)) { + return schedulableNameToSchedulable.get(schedulableName) } for (schedulable <- schedulableQueue) { val sched = schedulable.getSchedulableByName(schedulableName) @@ -95,11 +95,12 @@ private[spark] class Pool( shouldRevive } - override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + override def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] = { var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] - val sortedSchedulableQueue = schedulableQueue.sortWith(taskSetSchedulingAlgorithm.comparator) + val sortedSchedulableQueue = + schedulableQueue.toSeq.sortWith(taskSetSchedulingAlgorithm.comparator) for (schedulable <- sortedSchedulableQueue) { - sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue() + sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue } sortedTaskSetQueue } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index ed24eb6a549dd..a87ef030e69c2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import java.util.concurrent.ConcurrentLinkedQueue + import scala.collection.mutable.ArrayBuffer import org.apache.spark.scheduler.SchedulingMode.SchedulingMode @@ -28,7 +30,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode private[spark] trait Schedulable { var parent: Pool // child queues - def schedulableQueue: ArrayBuffer[Schedulable] + def schedulableQueue: ConcurrentLinkedQueue[Schedulable] def schedulingMode: SchedulingMode def weight: Int def minShare: Int @@ -42,5 +44,5 @@ private[spark] trait Schedulable { def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String): Unit def checkSpeculatableTasks(): Boolean - def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] + def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 5a68f38bc5844..ffd1d9432682b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -222,7 +222,7 @@ private[spark] class TaskSchedulerImpl( // Build a list of tasks to assign to each worker. val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = shuffledOffers.map(o => o.cores).toArray - val sortedTaskSets = rootPool.getSortedTaskSetQueue() + val sortedTaskSets = rootPool.getSortedTaskSetQueue for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index a8b605c5b212a..7532da88c6065 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -117,7 +117,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin } def resourceOffer(rootPool: Pool): Int = { - val taskSetQueue = rootPool.getSortedTaskSetQueue() + val taskSetQueue = rootPool.getSortedTaskSetQueue /* Just for Test*/ for (manager <- taskSetQueue) { logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( From 68f28dabe9c7679be82e684385be216319beb610 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 14 May 2014 04:17:32 -0700 Subject: [PATCH 52/82] Fixed streaming examples docs to use run-example instead of spark-submit Pretty self-explanatory Author: Tathagata Das Closes #722 from tdas/example-fix and squashes the following commits: 7839979 [Tathagata Das] Minor changes. 0673441 [Tathagata Das] Fixed java docs of java streaming example e687123 [Tathagata Das] Fixed scala style errors. 9b8d112 [Tathagata Das] Fixed streaming examples docs to use run-example instead of spark-submit. --- .../streaming/JavaCustomReceiver.java | 13 ++--- .../streaming/JavaFlumeEventCount.java | 6 +- .../streaming/JavaKafkaWordCount.java | 6 +- .../streaming/JavaNetworkWordCount.java | 13 +++-- .../examples/streaming/ActorWordCount.scala | 6 +- .../examples/streaming/CustomReceiver.scala | 19 +++--- .../examples/streaming/FlumeEventCount.scala | 9 ++- .../examples/streaming/HdfsWordCount.scala | 5 +- .../examples/streaming/KafkaWordCount.scala | 6 +- .../examples/streaming/MQTTWordCount.scala | 10 ++-- .../examples/streaming/NetworkWordCount.scala | 14 +++-- .../RecoverableNetworkWordCount.scala | 7 +-- .../streaming/StatefulNetworkWordCount.scala | 6 +- .../streaming/TwitterPopularTags.scala | 22 ++++++- .../examples/streaming/ZeroMQWordCount.scala | 8 +-- .../clickstream/PageViewGenerator.scala | 10 ++-- .../clickstream/PageViewStream.scala | 7 ++- .../twitter/TwitterInputDStream.scala | 58 +++++++++++-------- 18 files changed, 130 insertions(+), 95 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index 7f558f3ee713a..5622df5ce03ff 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -19,6 +19,7 @@ import com.google.common.collect.Lists; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; @@ -48,25 +49,23 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.examples.streaming.JavaCustomReceiver local[2] localhost 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.JavaCustomReceiver localhost 9999` */ public class JavaCustomReceiver extends Receiver { private static final Pattern SPACE = Pattern.compile(" "); public static void main(String[] args) { - if (args.length < 3) { - System.err.println("Usage: JavaNetworkWordCount \n" + - "In local mode, should be 'local[n]' with n > 1"); + if (args.length < 2) { + System.err.println("Usage: JavaNetworkWordCount "); System.exit(1); } StreamingExamples.setStreamingLogLevels(); // Create the context with a 1 second batch size - JavaStreamingContext ssc = new JavaStreamingContext(args[0], "JavaNetworkWordCount", - new Duration(1000), System.getenv("SPARK_HOME"), - JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); + SparkConf sparkConf = new SparkConf().setAppName("JavaCustomReceiver"); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); // Create a input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java index 400b68c2215b3..da56637fe891a 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java @@ -33,10 +33,12 @@ * Your Flume AvroSink should be pointed to this address. * * Usage: JavaFlumeEventCount - * * is the host the Flume receiver will be started on - a receiver * creates a server and listens for flume events. * is the port the Flume receiver will listen on. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.JavaFlumeEventCount ` */ public final class JavaFlumeEventCount { private JavaFlumeEventCount() { @@ -56,7 +58,7 @@ public static void main(String[] args) { Duration batchInterval = new Duration(2000); SparkConf sparkConf = new SparkConf().setAppName("JavaFlumeEventCount"); JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, batchInterval); - JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port); + JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, host, port); flumeStream.count(); diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java index 6a74cc50d19ed..16ae9a3319ee2 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java @@ -40,15 +40,15 @@ /** * Consumes messages from one or more topics in Kafka and does wordcount. + * * Usage: JavaKafkaWordCount * is a list of one or more zookeeper servers that make quorum * is the name of kafka consumer group * is a list of one or more kafka topics to consume from * is the number of threads the kafka consumer should use * - * Example: - * `./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.JavaKafkaWordCount zoo01,zoo02, \ + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.JavaKafkaWordCount zoo01,zoo02, \ * zoo03 my-consumer-group topic1,topic2 1` */ diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java index e5cbd39f437c2..45bcedebb4117 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java @@ -24,7 +24,7 @@ import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.examples.streaming.StreamingExamples; +import org.apache.spark.api.java.StorageLevels; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; @@ -41,8 +41,7 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.JavaNetworkWordCount localhost 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.JavaNetworkWordCount localhost 9999` */ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); @@ -54,13 +53,17 @@ public static void main(String[] args) { } StreamingExamples.setStreamingLogLevels(); - SparkConf sparkConf = new SparkConf().setAppName("JavaNetworkWordCount"); + // Create the context with a 1 second batch size + SparkConf sparkConf = new SparkConf().setAppName("JavaNetworkWordCount"); JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000)); // Create a JavaReceiverInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - JavaReceiverInputDStream lines = ssc.socketTextStream(args[0], Integer.parseInt(args[1])); + // Note that no duplication in storage level only for running locally. + // Replication necessary in distributed scenario for fault tolerance. + JavaReceiverInputDStream lines = ssc.socketTextStream( + args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index e29e16a9c1b17..b433082dce1a2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -130,11 +130,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` * and then run the example - * `./bin/spark-submit examples.jar --class org.apache.spark.examples.streaming.ActorWordCount \ - * 127.0.1.1 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index e317e2d36ae43..6bb659fbd8be8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.streaming import java.io.{InputStreamReader, BufferedReader, InputStream} import java.net.Socket -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ @@ -30,32 +30,27 @@ import org.apache.spark.streaming.receiver.Receiver * Custom Receiver that receives data over a socket. Received bytes is interpreted as * text and \n delimited lines are considered as records. They are then counted and printed. * - * Usage: CustomReceiver - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. - * and of the TCP server that Spark Streaming would connect to receive data. - * * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.examples.streaming.CustomReceiver local[2] localhost 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.CustomReceiver localhost 9999` */ object CustomReceiver { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: NetworkWordCount \n" + - "In local mode, should be 'local[n]' with n > 1") + if (args.length < 2) { + System.err.println("Usage: CustomReceiver ") System.exit(1) } StreamingExamples.setStreamingLogLevels() // Create the context with a 1 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) + val sparkConf = new SparkConf().setAppName("CustomReceiver") + val ssc = new StreamingContext(sparkConf, Seconds(1)) // Create a input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - val lines = ssc.receiverStream(new CustomReceiver(args(1), args(2).toInt)) + val lines = ssc.receiverStream(new CustomReceiver(args(0), args(1).toInt)) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala index 38362edac27f8..20e7df7c45b1b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala @@ -31,14 +31,16 @@ import org.apache.spark.util.IntParam * Your Flume AvroSink should be pointed to this address. * * Usage: FlumeEventCount - * * is the host the Flume receiver will be started on - a receiver * creates a server and listens for flume events. * is the port the Flume receiver will listen on. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.FlumeEventCount ` */ object FlumeEventCount { def main(args: Array[String]) { - if (args.length != 3) { + if (args.length < 2) { System.err.println( "Usage: FlumeEventCount ") System.exit(1) @@ -49,8 +51,9 @@ object FlumeEventCount { val Array(host, IntParam(port)) = args val batchInterval = Milliseconds(2000) - val sparkConf = new SparkConf().setAppName("FlumeEventCount") + // Create the context and set the batch size + val sparkConf = new SparkConf().setAppName("FlumeEventCount") val ssc = new StreamingContext(sparkConf, batchInterval) // Create a flume stream diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index 55ac48cfb6d10..6c24bc3ad09e0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -27,8 +27,9 @@ import org.apache.spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.HdfsWordCount localdir` + * $ bin/run-example \ + * org.apache.spark.examples.streaming.HdfsWordCount localdir + * * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 3af806981f37a..566ba6f911e02 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -35,9 +35,9 @@ import org.apache.spark.SparkConf * is the number of threads the kafka consumer should use * * Example: - * `./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.KafkaWordCount local[2] zoo01,zoo02,zoo03 \ - * my-consumer-group topic1,topic2 1` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.KafkaWordCount zoo01,zoo02,zoo03 \ + * my-consumer-group topic1,topic2 1` */ object KafkaWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index 3a10daa9ab84a..e4283e04a1b11 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -75,14 +75,14 @@ object MQTTPublisher { * Example Java code for Mqtt Publisher and Subscriber can be found here * https://bitbucket.org/mkjinesh/mqttclient * Usage: MQTTWordCount -\ * and describe where Mqtt publisher is running. + * and describe where Mqtt publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` * and run the example as - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.MQTTWordCount tcp://localhost:1883 foo` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.MQTTWordCount tcp://localhost:1883 foo` */ object MQTTWordCount { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index ad7a199b2c0ab..ae0a08c6cdb1a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -23,7 +23,7 @@ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.storage.StorageLevel /** - * Counts words in text encoded with UTF8 received from the network every second. + * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. * * Usage: NetworkWordCount * and describe the TCP server that Spark Streaming would connect to receive data. @@ -31,8 +31,7 @@ import org.apache.spark.storage.StorageLevel * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.NetworkWordCount localhost 9999` + * `$ bin/run-example org.apache.spark.examples.streaming.NetworkWordCount localhost 9999` */ object NetworkWordCount { def main(args: Array[String]) { @@ -42,13 +41,16 @@ object NetworkWordCount { } StreamingExamples.setStreamingLogLevels() - val sparkConf = new SparkConf().setAppName("NetworkWordCount"); + // Create the context with a 1 second batch size + val sparkConf = new SparkConf().setAppName("NetworkWordCount") val ssc = new StreamingContext(sparkConf, Seconds(1)) - // Create a NetworkInputDStream on target ip:port and count the + // Create a socket stream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_ONLY_SER) + // Note that no duplication in storage level only for running locally. + // Replication necessary in distributed scenario for fault tolerance. + val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index ace785d9fe4c5..6af3a0f33efc2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -46,8 +46,7 @@ import org.apache.spark.util.IntParam * * and run the example as * - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ + * `$ ./bin/run-example org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ * localhost 9999 ~/checkpoint/ ~/out` * * If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create @@ -57,7 +56,7 @@ import org.apache.spark.util.IntParam * * To run this example in a local standalone cluster with automatic driver recovery, * - * `$ ./spark-class org.apache.spark.deploy.Client -s launch \ + * `$ bin/spark-class org.apache.spark.deploy.Client -s launch \ * \ * org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ * localhost 9999 ~/checkpoint ~/out` @@ -81,7 +80,7 @@ object RecoverableNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(sparkConf, Seconds(1)) - // Create a NetworkInputDStream on target ip:port and count the + // Create a socket stream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') val lines = ssc.socketTextStream(ip, port) val words = lines.flatMap(_.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index 5e1415f3cc536..daa1ced63c701 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -31,8 +31,8 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/spark-submit examples.jar - * --class org.apache.spark.examples.streaming.StatefulNetworkWordCount localhost 9999` + * `$ bin/run-example + * org.apache.spark.examples.streaming.StatefulNetworkWordCount localhost 9999` */ object StatefulNetworkWordCount { def main(args: Array[String]) { @@ -51,7 +51,7 @@ object StatefulNetworkWordCount { Some(currentCount + previousCount) } - val sparkConf = new SparkConf().setAppName("NetworkWordCumulativeCountUpdateStateByKey") + val sparkConf = new SparkConf().setAppName("StatefulNetworkWordCount") // Create the context with a 1 second batch size val ssc = new StreamingContext(sparkConf, Seconds(1)) ssc.checkpoint(".") diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala index 1ddff22cb8a42..f55d23ab3924b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala @@ -28,13 +28,29 @@ import org.apache.spark.SparkConf * stream. The stream is instantiated with credentials and optionally filters supplied by the * command line arguments. * + * Run this on your local machine as + * */ object TwitterPopularTags { def main(args: Array[String]) { + if (args.length < 4) { + System.err.println("Usage: TwitterPopularTags " + + " []") + System.exit(1) + } StreamingExamples.setStreamingLogLevels() - val filters = args + val Array(consumerKey, consumerSecret, accessToken, accessTokenSecret) = args.take(4) + val filters = args.takeRight(args.length - 4) + + // Set the system properties so that Twitter4j library used by twitter stream + // can use them to generat OAuth credentials + System.setProperty("twitter4j.oauth.consumerKey", consumerKey) + System.setProperty("twitter4j.oauth.consumerSecret", consumerSecret) + System.setProperty("twitter4j.oauth.accessToken", accessToken) + System.setProperty("twitter4j.oauth.accessTokenSecret", accessTokenSecret) + val sparkConf = new SparkConf().setAppName("TwitterPopularTags") val ssc = new StreamingContext(sparkConf, Seconds(2)) val stream = TwitterUtils.createStream(ssc, None, filters) @@ -52,13 +68,13 @@ object TwitterPopularTags { // Print popular hashtags topCounts60.foreachRDD(rdd => { - val topList = rdd.take(5) + val topList = rdd.take(10) println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count())) topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} }) topCounts10.foreachRDD(rdd => { - val topList = rdd.take(5) + val topList = rdd.take(10) println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count())) topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} }) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index 7ade3f1018ee8..79905af381a12 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -68,11 +68,11 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./bin/spark-submit examples.jar \ - * --class org.apache.spark.examples.streaming.ZeroMQWordCount tcp://127.0.1.1:1234 foo` + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.ZeroMQWordCount tcp://127.0.1.1:1234 foo` */ // scalastyle:on object ZeroMQWordCount { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index 97e0cb92078dc..8402491b62671 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -40,11 +40,13 @@ object PageView extends Serializable { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * - * When running this, you may want to set the root logging level to ERROR in - * conf/log4j.properties to reduce the verbosity of the output. + * To run the generator + * `$ bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10` + * To process the generated stream + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444` + * */ // scalastyle:on object PageViewGenerator { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index d30ceffbe29cb..d9b886eff77cc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -26,8 +26,11 @@ import org.apache.spark.examples.streaming.StreamingExamples * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * To run the generator + * `$ bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10` + * To process the generated stream + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444` */ // scalastyle:on object PageViewStream { diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 7bca1407116fa..5ea2e5549d7df 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -63,36 +63,48 @@ class TwitterReceiver( storageLevel: StorageLevel ) extends Receiver[Status](storageLevel) with Logging { - var twitterStream: TwitterStream = _ + private var twitterStream: TwitterStream = _ def onStart() { - twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) - twitterStream.addListener(new StatusListener { - def onStatus(status: Status) = { - store(status) - } - // Unimplemented - def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} - def onTrackLimitationNotice(i: Int) {} - def onScrubGeo(l: Long, l1: Long) {} - def onStallWarning(stallWarning: StallWarning) {} - def onException(e: Exception) { - restart("Error receiving tweets", e) - } - }) + try { + val newTwitterStream = new TwitterStreamFactory().getInstance(twitterAuth) + newTwitterStream.addListener(new StatusListener { + def onStatus(status: Status) = { + store(status) + } + // Unimplemented + def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} + def onTrackLimitationNotice(i: Int) {} + def onScrubGeo(l: Long, l1: Long) {} + def onStallWarning(stallWarning: StallWarning) {} + def onException(e: Exception) { + restart("Error receiving tweets", e) + } + }) - val query = new FilterQuery - if (filters.size > 0) { - query.track(filters.toArray) - twitterStream.filter(query) - } else { - twitterStream.sample() + val query = new FilterQuery + if (filters.size > 0) { + query.track(filters.toArray) + newTwitterStream.filter(query) + } else { + newTwitterStream.sample() + } + setTwitterStream(newTwitterStream) + logInfo("Twitter receiver started") + } catch { + case e: Exception => restart("Error starting Twitter stream", e) } - logInfo("Twitter receiver started") } def onStop() { - twitterStream.shutdown() + setTwitterStream(null) logInfo("Twitter receiver stopped") } + + private def setTwitterStream(newTwitterStream: TwitterStream) = synchronized { + if (twitterStream != null) { + twitterStream.shutdown() + } + twitterStream = newTwitterStream + } } From 2e5a7cde223c8bf6d34e46b27ac94a965441584d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 14 May 2014 09:38:33 -0700 Subject: [PATCH 53/82] SPARK-1827. LICENSE and NOTICE files need a refresh to contain transitive dependency info LICENSE and NOTICE policy is explained here: http://www.apache.org/dev/licensing-howto.html http://www.apache.org/legal/3party.html This leads to the following changes. First, this change enables two extensions to maven-shade-plugin in assembly/ that will try to include and merge all NOTICE and LICENSE files. This can't hurt. This generates a consolidated NOTICE file that I manually added to NOTICE. Next, a list of all dependencies and their licenses was generated: `mvn ... license:aggregate-add-third-party` to create: `target/generated-sources/license/THIRD-PARTY.txt` Each dependency is listed with one or more licenses. Determine the most-compatible license for each if there is more than one. For "unknown" license dependencies, I manually evaluateD their license. Many are actually Apache projects or components of projects covered already. The only non-trivial one was Colt, which has its own (compatible) license. I ignored Apache-licensed and public domain dependencies as these require no further action (beyond NOTICE above). BSD and MIT licenses (permissive Category A licenses) are evidently supposed to be mentioned in LICENSE, so I added a section without output from the THIRD-PARTY.txt file appropriately. Everything else, Category B licenses, are evidently mentioned in NOTICE (?) Same there. LICENSE contained some license statements for source code that is redistributed. I left this as I think that is the right place to put it. Author: Sean Owen Closes #770 from srowen/SPARK-1827 and squashes the following commits: a764504 [Sean Owen] Add LICENSE and NOTICE info for all transitive dependencies as of 1.0 --- LICENSE | 103 +++++++++ NOTICE | 572 ++++++++++++++++++++++++++++++++++++++++++++++- assembly/pom.xml | 2 + 3 files changed, 671 insertions(+), 6 deletions(-) diff --git a/LICENSE b/LICENSE index 1c1c2c0255fa9..383f079df8c8b 100644 --- a/LICENSE +++ b/LICENSE @@ -428,3 +428,106 @@ LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON A THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +======================================================================== +For colt: +======================================================================== + +Copyright (c) 1999 CERN - European Organization for Nuclear Research. +Permission to use, copy, modify, distribute and sell this software and its documentation for any purpose is hereby granted without fee, provided that the above copyright notice appear in all copies and that both that copyright notice and this permission notice appear in supporting documentation. CERN makes no representations about the suitability of this software for any purpose. It is provided "as is" without expressed or implied warranty. + +Packages hep.aida.* + +Written by Pavel Binko, Dino Ferrero Merlino, Wolfgang Hoschek, Tony Johnson, Andreas Pfeiffer, and others. Check the FreeHEP home page for more info. Permission to use and/or redistribute this work is granted under the terms of the LGPL License, with the exception that any usage related to military applications is expressly forbidden. The software and documentation made available under the terms of this license are provided with no warranty. + + +======================================================================== +Fo SnapTree: +======================================================================== + +SNAPTREE LICENSE + +Copyright (c) 2009-2012 Stanford University, unless otherwise specified. +All rights reserved. + +This software was developed by the Pervasive Parallelism Laboratory of +Stanford University, California, USA. + +Permission to use, copy, modify, and distribute this software in source +or binary form for any purpose with or without fee is hereby granted, +provided that the following conditions are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + + 3. Neither the name of Stanford University nor the names of its + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + + +THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY +OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF +SUCH DAMAGE. + + +======================================================================== +BSD-style licenses +======================================================================== + +The following components are provided under a BSD-style license. See project link for details. + + (BSD 3 Clause) core (com.github.fommil.netlib:core:1.1.2 - https://github.com/fommil/netlib-java/core) + (BSD 3-clause style license) jblas (org.jblas:jblas:1.2.3 - http://jblas.org/) + (BSD License) AntLR Parser Generator (antlr:antlr:2.7.7 - http://www.antlr.org/) + (BSD License) Javolution (javolution:javolution:5.5.1 - http://javolution.org) + (BSD licence) ANTLR ST4 4.0.4 (org.antlr:ST4:4.0.4 - http://www.stringtemplate.org) + (BSD licence) ANTLR StringTemplate (org.antlr:stringtemplate:3.2.1 - http://www.stringtemplate.org) + (BSD style) Hamcrest Core (org.hamcrest:hamcrest-core:1.1 - no url defined) + (BSD) JLine (jline:jline:0.9.94 - http://jline.sourceforge.net) + (BSD) ParaNamer Core (com.thoughtworks.paranamer:paranamer:2.3 - http://paranamer.codehaus.org/paranamer) + (BSD) ParaNamer Core (com.thoughtworks.paranamer:paranamer:2.6 - http://paranamer.codehaus.org/paranamer) + (BSD-like) (The BSD License) jline (org.scala-lang:jline:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Actors library (org.scala-lang:scala-actors:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-compiler:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-reflect:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scala Library (org.scala-lang:scala-library:2.10.4 - http://www.scala-lang.org/) + (BSD-like) Scalap (org.scala-lang:scalap:2.10.4 - http://www.scala-lang.org/) + (BSD-style) scalacheck (org.scalacheck:scalacheck_2.10:1.10.0 - http://www.scalacheck.org) + (BSD-style) spire (org.spire-math:spire_2.10:0.7.1 - http://spire-math.org) + (BSD-style) spire-macros (org.spire-math:spire-macros_2.10:0.7.1 - http://spire-math.org) + (New BSD License) Kryo (com.esotericsoftware.kryo:kryo:2.21 - http://code.google.com/p/kryo/) + (New BSD License) MinLog (com.esotericsoftware.minlog:minlog:1.2 - http://code.google.com/p/minlog/) + (New BSD License) ReflectASM (com.esotericsoftware.reflectasm:reflectasm:1.07 - http://code.google.com/p/reflectasm/) + (New BSD license) Protocol Buffer Java API (com.google.protobuf:protobuf-java:2.5.0 - http://code.google.com/p/protobuf) + (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) + (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) + (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.8.1 - http://py4j.sourceforge.net/) + (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) + (ISC/BSD License) jbcrypt (org.mindrot:jbcrypt:0.3m - http://www.mindrot.org/) + +======================================================================== +MIT licenses +======================================================================== + +The following components are provided under the MIT License. See project link for details. + + (MIT License) JCL 1.1.1 implemented over SLF4J (org.slf4j:jcl-over-slf4j:1.7.5 - http://www.slf4j.org) + (MIT License) JUL to SLF4J bridge (org.slf4j:jul-to-slf4j:1.7.5 - http://www.slf4j.org) + (MIT License) SLF4J API Module (org.slf4j:slf4j-api:1.7.5 - http://www.slf4j.org) + (MIT License) SLF4J LOG4J-12 Binding (org.slf4j:slf4j-log4j12:1.7.5 - http://www.slf4j.org) + (MIT License) pyrolite (org.spark-project:pyrolite:2.0.1 - http://pythonhosted.org/Pyro4/) + (MIT License) scopt (com.github.scopt:scopt_2.10:3.2.0 - https://github.com/scopt/scopt) + (The MIT License) Mockito (org.mockito:mockito-all:1.8.5 - http://www.mockito.org) diff --git a/NOTICE b/NOTICE index 42f6c3a835725..452aef2871652 100644 --- a/NOTICE +++ b/NOTICE @@ -4,11 +4,571 @@ Copyright 2014 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). -In addition, this product includes: -- JUnit (http://www.junit.org) is a testing framework for Java. We included it - under the terms of the Eclipse Public License v1.0. +======================================================================== +Common Development and Distribution License 1.0 +======================================================================== -- JTransforms (https://sites.google.com/site/piotrwendykier/software/jtransforms) - provides fast transforms in Java. It is tri-licensed, and we included it under - the terms of the Mozilla Public License v1.1. +The following components are provided under the Common Development and Distribution License 1.0. See project link for details. + + (CDDL 1.0) Glassfish Jasper (org.mortbay.jetty:jsp-2.1:6.1.14 - http://jetty.mortbay.org/project/modules/jsp-2.1) + (CDDL 1.0) Servlet Specification 2.5 API (org.mortbay.jetty:servlet-api-2.5:6.1.14 - http://jetty.mortbay.org/project/modules/servlet-api-2.5) + (COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.0) (GNU General Public Library) Streaming API for XML (javax.xml.stream:stax-api:1.0-2 - no url defined) + (Common Development and Distribution License (CDDL) v1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp) + +======================================================================== +Common Development and Distribution License 1.1 +======================================================================== + +The following components are provided under the Common Development and Distribution License 1.1. See project link for details. + + (CDDL 1.1) (GPL2 w/ CPE) JAXB API bundle for GlassFish V3 (javax.xml.bind:jaxb-api:2.2.2 - https://jaxb.dev.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) JAXB RI (com.sun.xml.bind:jaxb-impl:2.2.3-1 - http://jaxb.java.net/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.8 - https://jersey.dev.java.net/jersey-core/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-core (com.sun.jersey:jersey-core:1.9 - https://jersey.java.net/jersey-core/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-guice (com.sun.jersey.contribs:jersey-guice:1.9 - https://jersey.java.net/jersey-contribs/jersey-guice/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.8 - https://jersey.dev.java.net/jersey-json/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-json (com.sun.jersey:jersey-json:1.9 - https://jersey.java.net/jersey-json/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.8 - https://jersey.dev.java.net/jersey-server/) + (CDDL 1.1) (GPL2 w/ CPE) jersey-server (com.sun.jersey:jersey-server:1.9 - https://jersey.java.net/jersey-server/) + +======================================================================== +Common Public License 1.0 +======================================================================== + +The following components are provided under the Common Public 1.0 License. See project link for details. + + (Common Public License Version 1.0) JUnit (junit:junit-dep:4.10 - http://junit.org) + (Common Public License Version 1.0) JUnit (junit:junit:3.8.1 - http://junit.org) + (Common Public License Version 1.0) JUnit (junit:junit:4.8.2 - http://junit.org) + +======================================================================== +Eclipse Public License 1.0 +======================================================================== + +The following components are provided under the Eclipse Public License 1.0. See project link for details. + + (Eclipse Public License - Version 1.0) mqtt-client (org.eclipse.paho:mqtt-client:0.4.0 - http://www.eclipse.org/paho/mqtt-client) + (Eclipse Public License v1.0) Eclipse JDT Core (org.eclipse.jdt:core:3.1.1 - http://www.eclipse.org/jdt/) + +======================================================================== +Mozilla Public License 1.0 +======================================================================== + +The following components are provided under the Mozilla Public License 1.0. See project link for details. + + (GPL) (LGPL) (MPL) JTransforms (com.github.rwl:jtransforms:2.4.0 - http://sourceforge.net/projects/jtransforms/) + (Mozilla Public License Version 1.1) jamon-runtime (org.jamon:jamon-runtime:2.3.1 - http://www.jamon.org/jamon-runtime/) + + + +======================================================================== +NOTICE files +======================================================================== + +The following NOTICEs are pertain to software distributed with this project. + + +// ------------------------------------------------------------------ +// NOTICE file corresponding to the section 4d of The Apache License, +// Version 2.0, in this case for +// ------------------------------------------------------------------ + +Apache Avro +Copyright 2009-2013 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Apache Commons Codec +Copyright 2002-2009 The Apache Software Foundation + +This product includes software developed by +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java contains +test data from http://aspell.sourceforge.net/test/batch0.tab. + +Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org). Verbatim copying +and distribution of this entire article is permitted in any medium, +provided this notice is preserved. +-------------------------------------------------------------------------------- + +Apache HttpComponents HttpClient +Copyright 1999-2011 The Apache Software Foundation + +This project contains annotations derived from JCIP-ANNOTATIONS +Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net + +Apache HttpComponents HttpCore +Copyright 2005-2011 The Apache Software Foundation + +Curator Recipes +Copyright 2011-2014 The Apache Software Foundation + +Curator Framework +Copyright 2011-2014 The Apache Software Foundation + +Curator Client +Copyright 2011-2014 The Apache Software Foundation + +Apache Geronimo +Copyright 2003-2008 The Apache Software Foundation + +Activation 1.1 +Copyright 2003-2007 The Apache Software Foundation + +Apache Commons Lang +Copyright 2001-2014 The Apache Software Foundation + +This product includes software from the Spring Framework, +under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + +Apache log4j +Copyright 2007 The Apache Software Foundation + +# Compress LZF + +This library contains efficient implementation of LZF compression format, +as well as additional helper classes that build on JDK-provided gzip (deflat) +codec. + +## Licensing + +Library is licensed under Apache License 2.0, as per accompanying LICENSE file. + +## Credit + +Library has been written by Tatu Saloranta (tatu.saloranta@iki.fi). +It was started at Ning, inc., as an official Open Source process used by +platform backend, but after initial versions has been developed outside of +Ning by supporting community. + +Other contributors include: + +* Jon Hartlaub (first versions of streaming reader/writer; unit tests) +* Cedrik Lime: parallel LZF implementation + +Various community members have contributed bug reports, and suggested minor +fixes; these can be found from file "VERSION.txt" in SCM. + +Objenesis +Copyright 2006-2009 Joe Walnes, Henri Tremblay, Leonardo Mesquita + +Apache Commons Net +Copyright 2001-2010 The Apache Software Foundation + + The Netty Project + ================= + +Please visit the Netty web site for more information: + + * http://netty.io/ + +Copyright 2011 The Netty Project + +The Netty Project licenses this file to you under the Apache License, +version 2.0 (the "License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, WITHOUT +WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the +License for the specific language governing permissions and limitations +under the License. + +Also, please refer to each LICENSE..txt file, which is located in +the 'license' directory of the distribution file, for the license terms of the +components that this product depends on. + +------------------------------------------------------------------------------- +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * license/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * license/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified version of 'JZlib', a re-implementation of +zlib in pure Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.jzlib.txt (BSD Style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product optionally depends on 'Protocol Buffers', Google's data +interchange format, which can be obtained at: + + * LICENSE: + * license/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * http://code.google.com/p/protobuf/ + +This product optionally depends on 'SLF4J', a simple logging facade for Java, +which can be obtained at: + + * LICENSE: + * license/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + +This product optionally depends on 'Apache Commons Logging', a logging +framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-logging.txt (Apache License 2.0) + * HOMEPAGE: + * http://commons.apache.org/logging/ + +This product optionally depends on 'Apache Log4J', a logging framework, +which can be obtained at: + + * LICENSE: + * license/LICENSE.log4j.txt (Apache License 2.0) + * HOMEPAGE: + * http://logging.apache.org/log4j/ + +This product optionally depends on 'JBoss Logging', a logging framework, +which can be obtained at: + + * LICENSE: + * license/LICENSE.jboss-logging.txt (GNU LGPL 2.1) + * HOMEPAGE: + * http://anonsvn.jboss.org/repos/common/common-logging-spi/ + +This product optionally depends on 'Apache Felix', an open source OSGi +framework implementation, which can be obtained at: + + * LICENSE: + * license/LICENSE.felix.txt (Apache License 2.0) + * HOMEPAGE: + * http://felix.apache.org/ + +This product optionally depends on 'Webbit', a Java event based +WebSocket and HTTP server: + + * LICENSE: + * license/LICENSE.webbit.txt (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +Jackson core and extension components may be licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +mesos +Copyright 2014 The Apache Software Foundation + +Apache Thrift +Copyright 2006-2010 The Apache Software Foundation. + + Apache Ant + Copyright 1999-2013 The Apache Software Foundation + + The task is based on code Copyright (c) 2002, Landmark + Graphics Corp that has been kindly donated to the Apache Software + Foundation. + +Apache Commons IO +Copyright 2002-2012 The Apache Software Foundation + +Apache Commons Math +Copyright 2001-2013 The Apache Software Foundation + +=============================================================================== + +The inverse error function implementation in the Erf class is based on CUDA +code developed by Mike Giles, Oxford-Man Institute of Quantitative Finance, +and published in GPU Computing Gems, volume 2, 2010. +=============================================================================== + +The BracketFinder (package org.apache.commons.math3.optimization.univariate) +and PowellOptimizer (package org.apache.commons.math3.optimization.general) +classes are based on the Python code in module "optimize.py" (version 0.5) +developed by Travis E. Oliphant for the SciPy library (http://www.scipy.org/) +Copyright © 2003-2009 SciPy Developers. +=============================================================================== + +The LinearConstraint, LinearObjectiveFunction, LinearOptimizer, +RelationShip, SimplexSolver and SimplexTableau classes in package +org.apache.commons.math3.optimization.linear include software developed by +Benjamin McCann (http://www.benmccann.com) and distributed with +the following copyright: Copyright 2009 Google Inc. +=============================================================================== + +This product includes software developed by the +University of Chicago, as Operator of Argonne National +Laboratory. +The LevenbergMarquardtOptimizer class in package +org.apache.commons.math3.optimization.general includes software +translated from the lmder, lmpar and qrsolv Fortran routines +from the Minpack package +Minpack Copyright Notice (1999) University of Chicago. All rights reserved +=============================================================================== + +The GraggBulirschStoerIntegrator class in package +org.apache.commons.math3.ode.nonstiff includes software translated +from the odex Fortran routine developed by E. Hairer and G. Wanner. +Original source copyright: +Copyright (c) 2004, Ernst Hairer +=============================================================================== + +The EigenDecompositionImpl class in package +org.apache.commons.math3.linear includes software translated +from some LAPACK Fortran routines. Original source copyright: +Copyright (c) 1992-2008 The University of Tennessee. All rights reserved. +=============================================================================== + +The MersenneTwister class in package org.apache.commons.math3.random +includes software translated from the 2002-01-26 version of +the Mersenne-Twister generator written in C by Makoto Matsumoto and Takuji +Nishimura. Original source copyright: +Copyright (C) 1997 - 2002, Makoto Matsumoto and Takuji Nishimura, +All rights reserved +=============================================================================== + +The LocalizedFormatsTest class in the unit tests is an adapted version of +the OrekitMessagesTest class from the orekit library distributed under the +terms of the Apache 2 licence. Original source copyright: +Copyright 2010 CS Systèmes d'Information +=============================================================================== + +The HermiteInterpolator class and its corresponding test have been imported from +the orekit library distributed under the terms of the Apache 2 licence. Original +source copyright: +Copyright 2010-2012 CS Systèmes d'Information +=============================================================================== + +The creation of the package "o.a.c.m.analysis.integration.gauss" was inspired +by an original code donated by Sébastien Brisard. +=============================================================================== + +The complete text of licenses and disclaimers associated with the the original +sources enumerated above at the time of code translation are in the LICENSE.txt +file. + +This product currently only contains code developed by authors +of specific components, as identified by the source code files; +if such notes are missing files have been created by +Tatu Saloranta. + +For additional credits (generally to people who reported problems) +see CREDITS file. + +Apache Commons Lang +Copyright 2001-2011 The Apache Software Foundation + +Apache Commons Compress +Copyright 2002-2012 The Apache Software Foundation + +Apache Commons CLI +Copyright 2001-2009 The Apache Software Foundation + +Google Guice - Extensions - Servlet +Copyright 2006-2011 Google, Inc. + +Google Guice - Core Library +Copyright 2006-2011 Google, Inc. + +Apache Jakarta HttpClient +Copyright 1999-2007 The Apache Software Foundation + +Apache Hive +Copyright 2008-2013 The Apache Software Foundation + +This product includes software developed by The Apache Software +Foundation (http://www.apache.org/). + +This product includes software developed by The JDBM Project +(http://jdbm.sourceforge.net/). + +This product includes/uses ANTLR (http://www.antlr.org/), +Copyright (c) 2003-2011, Terrence Parr. + +This product includes/uses StringTemplate (http://www.stringtemplate.org/), +Copyright (c) 2011, Terrence Parr. + +This product includes/uses ASM (http://asm.ow2.org/), +Copyright (c) 2000-2007 INRIA, France Telecom. + +This product includes/uses org.json (http://www.json.org/java/index.html), +Copyright (c) 2002 JSON.org + +This product includes/uses JLine (http://jline.sourceforge.net/), +Copyright (c) 2002-2006, Marc Prud'hommeaux . + +This product includes/uses SQLLine (http://sqlline.sourceforge.net), +Copyright (c) 2002, 2003, 2004, 2005 Marc Prud'hommeaux . + +This product includes/uses SLF4J (http://www.slf4j.org/), +Copyright (c) 2004-2010 QOS.ch + +This product includes/uses Bootstrap (http://twitter.github.com/bootstrap/), +Copyright (c) 2012 Twitter, Inc. + +This product includes/uses Glyphicons (http://glyphicons.com/), +Copyright (c) 2010 - 2012 Jan Kovarík + +This product includes DataNucleus (http://www.datanucleus.org/) +Copyright 2008-2008 DataNucleus + +This product includes Guava (http://code.google.com/p/guava-libraries/) +Copyright (C) 2006 Google Inc. + +This product includes JavaEWAH (http://code.google.com/p/javaewah/) +Copyright (C) 2011 Google Inc. + +Apache Commons Pool +Copyright 1999-2009 The Apache Software Foundation + +========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the DataNucleus distribution. == +========================================================================= + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Erik Bengtson +Andy Jefferson + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Joerg von Frantzius +Thomas Marti +Barry Haddow +Marco Schulze +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Marcus Mennemeier +Xuan Baldauf +Eric Sultan + +=================================================================== +This product also includes software developed by the TJDO project +(http://tjdo.sourceforge.net/). +=================================================================== + +=================================================================== +This product includes software developed by many individuals, +including the following: +=================================================================== +Andy Jefferson +Erik Bengtson +Joerg von Frantzius +Marco Schulze + +=================================================================== +This product has included contributions from some individuals, +including the following: +=================================================================== +Barry Haddow +Ralph Ullrich +David Ezzio +Brendan de Beer +David Eaves +Martin Taal +Tony Lai +Roland Szabo +Anton Troshin (Timesten) + +=================================================================== +This product also includes software developed by the Apache Commons project +(http://commons.apache.org/). +=================================================================== + +Apache Java Data Objects (JDO) +Copyright 2005-2006 The Apache Software Foundation + +========================================================================= +== NOTICE file corresponding to section 4(d) of the Apache License, == +== Version 2.0, in this case for the Apache Derby distribution. == +========================================================================= + +Apache Derby +Copyright 2004-2008 The Apache Software Foundation + +Portions of Derby were originally developed by +International Business Machines Corporation and are +licensed to the Apache Software Foundation under the +"Software Grant and Corporate Contribution License Agreement", +informally known as the "Derby CLA". +The following copyright notice(s) were affixed to portions of the code +with which this file is now or was at one time distributed +and are placed here unaltered. + +(C) Copyright 1997,2004 International Business Machines Corporation. All rights reserved. + +(C) Copyright IBM Corp. 2003. + +The portion of the functionTests under 'nist' was originally +developed by the National Institute of Standards and Technology (NIST), +an agency of the United States Department of Commerce, and adapted by +International Business Machines Corporation in accordance with the NIST +Software Acknowledgment and Redistribution document at +http://www.itl.nist.gov/div897/ctg/sql_form.htm + +Apache Commons Collections +Copyright 2001-2008 The Apache Software Foundation + +Apache Commons Configuration +Copyright 2001-2008 The Apache Software Foundation + +Apache Jakarta Commons Digester +Copyright 2001-2006 The Apache Software Foundation + +Apache Commons BeanUtils +Copyright 2000-2008 The Apache Software Foundation + +Apache Avro Mapred API +Copyright 2009-2013 The Apache Software Foundation + +Apache Avro IPC +Copyright 2009-2013 The Apache Software Foundation diff --git a/assembly/pom.xml b/assembly/pom.xml index 6c4d46aeb67bd..abd8935339992 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -122,6 +122,8 @@ log4j.properties + + From d1d41ccee49a5c093cb61c791c01f64f2076b83e Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 14 May 2014 09:45:33 -0700 Subject: [PATCH 54/82] SPARK-1818 Freshen Mesos documentation Place more emphasis on using precompiled binary versions of Spark and Mesos instead of encouraging the reader to compile from source. Author: Andrew Ash Closes #756 from ash211/spark-1818 and squashes the following commits: 7ef3b33 [Andrew Ash] Brief explanation of the interactions between Spark and Mesos e7dea8e [Andrew Ash] Add troubleshooting and debugging section 956362d [Andrew Ash] Don't need to pass spark.executor.uri into the spark shell de3353b [Andrew Ash] Wrap to 100char 7ebf6ef [Andrew Ash] Polish on the section on Mesos Master URLs 3dcc2c1 [Andrew Ash] Use --tgz parameter of make-distribution 41b68ed [Andrew Ash] Period at end of sentence; formatting on :5050 8bf2c53 [Andrew Ash] Update site.MESOS_VERSIOn to match /pom.xml 74f2040 [Andrew Ash] SPARK-1818 Freshen Mesos documentation --- docs/_config.yml | 2 +- docs/running-on-mesos.md | 200 +++++++++++++++++++++++++++++++++------ 2 files changed, 174 insertions(+), 28 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index d177e38f884ca..45b78fe724a50 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,6 +7,6 @@ SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.13.0 +MESOS_VERSION: 0.18.1 SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 68259f0cb8ad1..ef762aa7b8fcc 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -3,19 +3,123 @@ layout: global title: Running Spark on Mesos --- -Spark can run on clusters managed by [Apache Mesos](http://mesos.apache.org/). Follow the steps below to install Mesos and Spark: - -1. Download and build Spark using the instructions [here](index.html). **Note:** Don't forget to consider what version of HDFS you might want to use! -2. Download, build, install, and start Mesos {{site.MESOS_VERSION}} on your cluster. You can download the Mesos distribution from a [mirror](http://www.apache.org/dyn/closer.cgi/mesos/{{site.MESOS_VERSION}}/). See the Mesos [Getting Started](http://mesos.apache.org/gettingstarted) page for more information. **Note:** If you want to run Mesos without installing it into the default paths on your system (e.g., if you don't have administrative privileges to install it), you should also pass the `--prefix` option to `configure` to tell it where to install. For example, pass `--prefix=/home/user/mesos`. By default the prefix is `/usr/local`. -3. Create a Spark "distribution" using `make-distribution.sh`. -4. Rename the `dist` directory created from `make-distribution.sh` to `spark-{{site.SPARK_VERSION}}`. -5. Create a `tar` archive: `tar czf spark-{{site.SPARK_VERSION}}.tar.gz spark-{{site.SPARK_VERSION}}` -6. Upload this archive to HDFS or another place accessible from Mesos via `http://`, e.g., [Amazon Simple Storage Service](http://aws.amazon.com/s3): `hadoop fs -put spark-{{site.SPARK_VERSION}}.tar.gz /path/to/spark-{{site.SPARK_VERSION}}.tar.gz` -7. Create a file called `spark-env.sh` in Spark's `conf` directory, by copying `conf/spark-env.sh.template`, and add the following lines to it: - * `export MESOS_NATIVE_LIBRARY=`. This path is usually `/lib/libmesos.so` (where the prefix is `/usr/local` by default, see above). Also, on Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`. - * `export SPARK_EXECUTOR_URI=`. - * `export MASTER=mesos://HOST:PORT` where HOST:PORT is the host and port (default: 5050) of your Mesos master (or `zk://...` if using Mesos with ZooKeeper). -8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the master URL. In addition, you'll need to set the `spark.executor.uri` property. For example: +# Why Mesos + +Spark can run on hardware clusters managed by [Apache Mesos](http://mesos.apache.org/). + +The advantages of deploying Spark with Mesos include: +- dynamic partitioning between Spark and other + [frameworks](https://mesos.apache.org/documentation/latest/mesos-frameworks/) +- scalable partitioning between multiple instances of Spark + +# How it works + +In a standalone cluster deployment, the cluster manager in the below diagram is a Spark master +instance. When using Mesos, the Mesos master replaces the Spark master as the cluster manager. + +

+ Spark cluster components +

+ +Now when a driver creates a job and starts issuing tasks for scheduling, Mesos determines what +machines handle what tasks. Because it takes into account other frameworks when scheduling these +many short-lived tasks, multiple frameworks can coexist on the same cluster without resorting to a +static partitioning of resources. + +To get started, follow the steps below to install Mesos and deploy Spark jobs via Mesos. + + +# Installing Mesos + +Spark {{site.SPARK_VERSION}} is designed for use with Mesos {{site.MESOS_VERSION}} and does not +require any special patches of Mesos. + +If you already have a Mesos cluster running, you can skip this Mesos installation step. + +Otherwise, installing Mesos for Spark is no different than installing Mesos for use by other +frameworks. You can install Mesos using either prebuilt packages or by compiling from source. + +## Prebuilt packages + +The Apache Mesos project only publishes source package releases, no binary releases. But other +third party projects publish binary releases that may be helpful in setting Mesos up. + +One of those is Mesosphere. To install Mesos using the binary releases provided by Mesosphere: + +1. Download Mesos installation package from [downloads page](http://mesosphere.io/downloads/) +2. Follow their instructions for installation and configuration + +The Mesosphere installation documents suggest setting up ZooKeeper to handle Mesos master failover, +but Mesos can be run without ZooKeeper using a single master as well. + +## From source + +To install Mesos directly from the upstream project rather than a third party, install from source. + +1. Download the Mesos distribution from a + [mirror](http://www.apache.org/dyn/closer.cgi/mesos/{{site.MESOS_VERSION}}/) +2. Follow the Mesos [Getting Started](http://mesos.apache.org/gettingstarted) page for compiling and + installing Mesos + +**Note:** If you want to run Mesos without installing it into the default paths on your system +(e.g., if you lack administrative privileges to install it), you should also pass the +`--prefix` option to `configure` to tell it where to install. For example, pass +`--prefix=/home/user/mesos`. By default the prefix is `/usr/local`. + +## Verification + +To verify that the Mesos cluster is ready for Spark, navigate to the Mesos master webui at port +`:5050` Confirm that all expected machines are present in the slaves tab. + + +# Connecting Spark to Mesos + +To use Mesos from Spark, you need a Spark distribution available in a place accessible by Mesos, and +a Spark driver program configured to connect to Mesos. + +## Uploading Spark Distribution + +When Mesos runs a task on a Mesos slave for the first time, that slave must have a distribution of +Spark available for running the Spark Mesos executor backend. A distribution of Spark is just a +compiled binary version of Spark. + +The Spark distribution can be hosted at any Hadoop URI, including HTTP via `http://`, [Amazon Simple +Storage Service](http://aws.amazon.com/s3) via `s3://`, or HDFS via `hdfs:///`. + +To use a precompiled distribution: + +1. Download a Spark distribution from the Spark [download page](https://spark.apache.org/downloads.html) +2. Upload to hdfs/http/s3 + +To host on HDFS, use the Hadoop fs put command: `hadoop fs -put spark-{{site.SPARK_VERSION}}.tar.gz +/path/to/spark-{{site.SPARK_VERSION}}.tar.gz` + + +Or if you are using a custom-compiled version of Spark, you will need to create a distribution using +the `make-distribution.sh` script included in a Spark source tarball/checkout. + +1. Download and build Spark using the instructions [here](index.html) +2. Create a Spark distribution using `make-distribution.sh --tgz`. +3. Upload archive to http/s3/hdfs + + +## Using a Mesos Master URL + +The Master URLs for Mesos are in the form `mesos://host:5050` for a single-master Mesos +cluster, or `zk://host:2181` for a multi-master Mesos cluster using ZooKeeper. + +The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: + +1. In `spark.env.sh` set some environment variables: + * `export MESOS_NATIVE_LIBRARY=`. This path is typically + `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation + instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of + `libmesos.so`. + * `export SPARK_EXECUTOR_URI=`. +2. Also set `spark.executor.uri` to + +Now when starting a Spark application against the cluster, pass a `mesos://` +or `zk://` URL as the master when creating a `SparkContext`. For example: {% highlight scala %} val conf = new SparkConf() @@ -25,31 +129,73 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} +When running a shell the `spark.executor.uri` parameter is inherited from `SPARK_EXECUTOR_URI`, so +it does not need to be redundantly passed in as a system property. + +{% highlight bash %} +./bin/spark-shell --master mesos://host:5050 +{% endhighlight %} + + # Mesos Run Modes -Spark can run over Mesos in two modes: "fine-grained" and "coarse-grained". In fine-grained mode, which is the default, -each Spark task runs as a separate Mesos task. This allows multiple instances of Spark (and other frameworks) to share -machines at a very fine granularity, where each application gets more or fewer machines as it ramps up, but it comes with an -additional overhead in launching each task, which may be inappropriate for low-latency applications (e.g. interactive queries or serving web requests). The coarse-grained mode will instead -launch only *one* long-running Spark task on each Mesos machine, and dynamically schedule its own "mini-tasks" within -it. The benefit is much lower startup overhead, but at the cost of reserving the Mesos resources for the complete duration -of the application. +Spark can run over Mesos in two modes: "fine-grained" (default) and "coarse-grained". + +In "fine-grained" mode (default), each Spark task runs as a separate Mesos task. This allows +multiple instances of Spark (and other frameworks) to share machines at a very fine granularity, +where each application gets more or fewer machines as it ramps up and down, but it comes with an +additional overhead in launching each task. This mode may be inappropriate for low-latency +requirements like interactive queries or serving web requests. + +The "coarse-grained" mode will instead launch only *one* long-running Spark task on each Mesos +machine, and dynamically schedule its own "mini-tasks" within it. The benefit is much lower startup +overhead, but at the cost of reserving the Mesos resources for the complete duration of the +application. -To run in coarse-grained mode, set the `spark.mesos.coarse` property in your [SparkConf](configuration.html#spark-properties): +To run in coarse-grained mode, set the `spark.mesos.coarse` property in your +[SparkConf](configuration.html#spark-properties): {% highlight scala %} conf.set("spark.mesos.coarse", "true") {% endhighlight %} -In addition, for coarse-grained mode, you can control the maximum number of resources Spark will acquire. By default, -it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just one -application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). +In addition, for coarse-grained mode, you can control the maximum number of resources Spark will +acquire. By default, it will acquire *all* cores in the cluster (that get offered by Mesos), which +only makes sense if you run just one application at a time. You can cap the maximum number of cores +using `conf.set("spark.cores.max", "10")` (for example). # Running Alongside Hadoop -You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a separate service on the machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). +You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a +separate service on the machines. To access Hadoop data from Spark, a full hdfs:// URL is required +(typically `hdfs://:9000/path`, but you can find the right URL on your Hadoop Namenode web +UI). + +In addition, it is possible to also run Hadoop MapReduce on Mesos for better resource isolation and +sharing between the two. In this case, Mesos will act as a unified scheduler that assigns cores to +either Hadoop or Spark, as opposed to having them share resources via the Linux scheduler on each +node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop). + +In either case, HDFS runs separately from Hadoop MapReduce, without being scheduled through Mesos. + + +# Troubleshooting and Debugging + +A few places to look during debugging: + +- Mesos master on port `:5050` + - Slaves should appear in the slaves tab + - Spark applications should appear in the frameworks tab + - Tasks should appear in the details of a framework + - Check the stdout and stderr of the sandbox of failed tasks +- Mesos logs + - Master and slave logs are both in `/var/log/mesos` by default -In addition, it is possible to also run Hadoop MapReduce on Mesos, to get better resource isolation and sharing between the two. In this case, Mesos will act as a unified scheduler that assigns cores to either Hadoop or Spark, as opposed to having them share resources via the Linux scheduler on each node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop). +And common pitfalls: -In either case, HDFS runs separately from Hadoop MapReduce, without going through Mesos. +- Spark assembly not reachable/accessible + - Slaves need to be able to download the distribution +- Firewall blocking communications + - Check for messages about failed connections + - Temporarily disable firewalls for debugging and then poke appropriate holes From d58cb33ffa9e98a64cecea7b40ce7bfbed145079 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 14 May 2014 09:51:01 -0700 Subject: [PATCH 55/82] SPARK-1828: Created forked version of hive-exec that doesn't bundle other dependencies See https://issues.apache.org/jira/browse/SPARK-1828 for more information. This is being submitted to Jenkin's for testing. The dependency won't fully propagate in Maven central for a few more hours. Author: Patrick Wendell Closes #767 from pwendell/hive-shaded and squashes the following commits: ea10ac5 [Patrick Wendell] SPARK-1828: Created forked version of hive-exec that doesn't bundle other dependencies --- project/SparkBuild.scala | 6 +++--- sql/hive/pom.xml | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 57b3e22f81713..8d56b40431388 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -489,9 +489,9 @@ object SparkBuild extends Build { name := "spark-hive", javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( - "org.apache.hive" % "hive-metastore" % hiveVersion, - "org.apache.hive" % "hive-exec" % hiveVersion, - "org.apache.hive" % "hive-serde" % hiveVersion + "org.spark-project.hive" % "hive-metastore" % hiveVersion, + "org.spark-project.hive" % "hive-exec" % hiveVersion, + "org.spark-project.hive" % "hive-serde" % hiveVersion ), // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 8b32451d76045..9254b70e64a08 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -44,12 +44,12 @@ ${project.version}
- org.apache.hive + org.spark-project.hive hive-metastore ${hive.version} - org.apache.hive + org.spark-project.hive hive-exec ${hive.version} @@ -64,7 +64,7 @@ jackson-mapper-asl - org.apache.hive + org.spark-project.hive hive-serde ${hive.version} From 17f3075bc4aa8cbed165f7b367f70e84b1bc8db9 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 14 May 2014 10:07:25 -0700 Subject: [PATCH 56/82] [SPARK-1620] Handle uncaught exceptions in function run by Akka scheduler If the intended behavior was that uncaught exceptions thrown in functions being run by the Akka scheduler would end up being handled by the default uncaught exception handler set in Executor, and if that behavior is, in fact, correct, then this is a way to accomplish that. I'm not certain, though, that we shouldn't be doing something different to handle uncaught exceptions from some of these scheduled functions. In any event, this PR covers all of the cases I comment on in [SPARK-1620](https://issues.apache.org/jira/browse/SPARK-1620). Author: Mark Hamstra Closes #622 from markhamstra/SPARK-1620 and squashes the following commits: 071d193 [Mark Hamstra] refactored post-SPARK-1772 1a6a35e [Mark Hamstra] another style fix d30eb94 [Mark Hamstra] scalastyle 3573ecd [Mark Hamstra] Use wrapped try/catch in Utils.tryOrExit 8fc0439 [Mark Hamstra] Make functions run by the Akka scheduler use Executor's UncaughtExceptionHandler --- .../apache/spark/deploy/client/AppClient.scala | 18 ++++++++++-------- .../apache/spark/deploy/worker/Worker.scala | 18 ++++++++++-------- .../spark/scheduler/TaskSchedulerImpl.scala | 3 ++- .../apache/spark/storage/BlockManager.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 13 +++++++++++++ 5 files changed, 36 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 896913d796604..d38e9e79204c2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -30,7 +30,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} /** * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, @@ -88,13 +88,15 @@ private[spark] class AppClient( var retries = 0 registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - retries += 1 - if (registered) { - registrationRetryTimer.foreach(_.cancel()) - } else if (retries >= REGISTRATION_RETRIES) { - markDead("All masters are unresponsive! Giving up.") - } else { - tryRegisterAllMasters() + Utils.tryOrExit { + retries += 1 + if (registered) { + registrationRetryTimer.foreach(_.cancel()) + } else if (retries >= REGISTRATION_RETRIES) { + markDead("All masters are unresponsive! Giving up.") + } else { + tryRegisterAllMasters() + } } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 85d25dc7dbfa4..134624c35a57e 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -166,14 +166,16 @@ private[spark] class Worker( var retries = 0 registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - retries += 1 - if (registered) { - registrationRetryTimer.foreach(_.cancel()) - } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - System.exit(1) - } else { - tryRegisterAllMasters() + Utils.tryOrExit { + retries += 1 + if (registered) { + registrationRetryTimer.foreach(_.cancel()) + } else if (retries >= REGISTRATION_RETRIES) { + logError("All masters are unresponsive! Giving up.") + System.exit(1) + } else { + tryRegisterAllMasters() + } } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index ffd1d9432682b..649eed213e09a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -31,6 +31,7 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.util.Utils /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. @@ -139,7 +140,7 @@ private[spark] class TaskSchedulerImpl( import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, SPECULATION_INTERVAL milliseconds) { - checkSpeculatableTasks() + Utils.tryOrExit { checkSpeculatableTasks() } } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 6d7d4f922e1fa..6534095811907 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -155,7 +155,7 @@ private[spark] class BlockManager( BlockManagerWorker.startBlockManagerWorker(this) if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) { - heartBeat() + Utils.tryOrExit { heartBeat() } } } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 99ef6dd1fa6e3..d041bfa66a1cd 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -40,6 +40,7 @@ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.ExecutorUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** @@ -780,6 +781,18 @@ private[spark] object Utils extends Logging { output.toString } + /** + * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the + * default UncaughtExceptionHandler + */ + def tryOrExit(block: => Unit) { + try { + block + } catch { + case t: Throwable => ExecutorUncaughtExceptionHandler.uncaughtException(t) + } + } + /** * A regular expression to match classes of the "core" Spark API that we want to skip when * finding the call site of a method. From fde82c1549c78f1eebbb21ec34e60befbbff65f5 Mon Sep 17 00:00:00 2001 From: witgo Date: Wed, 14 May 2014 11:19:26 -0700 Subject: [PATCH 57/82] Fix: sbt test throw an java.lang.OutOfMemoryError: PermGen space Author: witgo Closes #773 from witgo/sbt_javaOptions and squashes the following commits: 26c7d38 [witgo] Improve sbt configuration --- .rat-excludes | 5 +++++ project/SparkBuild.scala | 1 + 2 files changed, 6 insertions(+) diff --git a/.rat-excludes b/.rat-excludes index 50766954ef070..689467847d80d 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -43,3 +43,8 @@ test.out/* .*iml service.properties db.lck +build/* +dist/* +.*out +.*ipr +.*iws diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8d56b40431388..6adec556322ba 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -183,6 +183,7 @@ object SparkBuild extends Build { javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark").map { case (k,v) => s"-D$k=$v" }.toSeq, + javaOptions in Test ++= "-Xmx3g -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g".split(" ").toSeq, javaOptions += "-Xmx3g", // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), From a3315d7f4c7584dae2ee0aa33c6ec9e97b229b48 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 14 May 2014 12:01:14 -0700 Subject: [PATCH 58/82] SPARK-1829 Sub-second durations shouldn't round to "0 s" As "99 ms" up to 99 ms As "0.1 s" from 0.1 s up to 0.9 s https://issues.apache.org/jira/browse/SPARK-1829 Compare the first image to the second here: http://imgur.com/RaLEsSZ,7VTlgfo#0 Author: Andrew Ash Closes #768 from ash211/spark-1829 and squashes the following commits: 1c15b8e [Andrew Ash] SPARK-1829 Format sub-second durations more appropriately --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a3d6a1821245b..a43314f48112f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -36,7 +36,13 @@ private[spark] object UIUtils extends Logging { def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) def formatDuration(milliseconds: Long): String = { + if (milliseconds < 100) { + return "%d ms".format(milliseconds) + } val seconds = milliseconds.toDouble / 1000 + if (seconds < 1) { + return "%.1f s".format(seconds) + } if (seconds < 60) { return "%.0f s".format(seconds) } From 65533c7ec03e7eedf5cd9756822863ab6f034ec9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 14 May 2014 12:53:30 -0700 Subject: [PATCH 59/82] SPARK-1833 - Have an empty SparkContext constructor. This is nicer than relying on new SparkContext(new SparkConf()) Author: Patrick Wendell Closes #774 from pwendell/spark-context and squashes the following commits: ef9f12f [Patrick Wendell] SPARK-1833 - Have an empty SparkContext constructor. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 032b3d744c619..634c10c33fc86 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -66,6 +66,12 @@ class SparkContext(config: SparkConf) extends Logging { // contains a map from hostname to a list of input format splits on the host. private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + /** + * Create a SparkContext that loads settings from system properties (for instance, when + * launching with ./bin/spark-submit). + */ + def this() = this(new SparkConf()) + /** * :: DeveloperApi :: * Alternative constructor for setting preferred locations where Spark will create executors. From 94c6c06ea13032b80610b3f54401d2ef2aa4874a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 14 May 2014 14:57:17 -0700 Subject: [PATCH 60/82] [FIX] do not load defaults when testing SparkConf in pyspark The default constructor loads default properties, which can fail the test. Author: Xiangrui Meng Closes #775 from mengxr/pyspark-conf-fix and squashes the following commits: 83ef6c4 [Xiangrui Meng] do not load defaults when testing SparkConf in pyspark --- python/pyspark/conf.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 49b68d57ab02c..8eff4a242a529 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -33,7 +33,7 @@ >>> sc.sparkHome == None True ->>> conf = SparkConf() +>>> conf = SparkConf(loadDefaults=False) >>> conf.setSparkHome("/path") >>> conf.get("spark.home") From 601e37198b97ba52e72ac13213c391c932e97b67 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Wed, 14 May 2014 15:45:52 -0700 Subject: [PATCH 61/82] String interpolation + some other small changes After having been invited to make the change in https://github.com/apache/spark/commit/6bee01dd04ef73c6b829110ebcdd622d521ea8ff#commitcomment-6284165 by @witgo. Author: Jacek Laskowski Closes #748 from jaceklaskowski/sparkenv-string-interpolation and squashes the following commits: be6ebac [Jacek Laskowski] String interpolation + some other small changes --- .../main/scala/org/apache/spark/SparkEnv.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 19d507c0cf860..720151a6b0f84 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -278,10 +278,11 @@ object SparkEnv extends Logging { addedJars: Seq[String], addedFiles: Seq[String]): Map[String, Seq[(String, String)]] = { + import Properties._ val jvmInformation = Seq( - ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), - ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString) + ("Java Version", s"$javaVersion ($javaVendor)"), + ("Java Home", javaHome), + ("Scala Version", versionString) ).sorted // Spark properties @@ -296,18 +297,15 @@ object SparkEnv extends Logging { // System properties that are not java classpaths val systemProperties = System.getProperties.iterator.toSeq - val otherProperties = systemProperties.filter { case (k, v) => + val otherProperties = systemProperties.filter { case (k, _) => k != "java.class.path" && !k.startsWith("spark.") }.sorted // Class paths including all added jars and files - val classPathProperty = systemProperties.find { case (k, v) => - k == "java.class.path" - }.getOrElse(("", "")) - val classPathEntries = classPathProperty._2 + val classPathEntries = javaClassPath .split(File.pathSeparator) - .filterNot(e => e.isEmpty) - .map(e => (e, "System Classpath")) + .filterNot(_.isEmpty) + .map((_, "System Classpath")) val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted From e3d72a74ad007c2bf279d6a74cdaca948bdf0ddd Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 14 May 2014 17:18:30 -0700 Subject: [PATCH 62/82] [SPARK-1696][MLLIB] use alpha in dense dspr It doesn't affect existing code because only `alpha = 1.0` is used in the code. Author: Xiangrui Meng Closes #778 from mengxr/mllib-dspr-fix and squashes the following commits: a37402e [Xiangrui Meng] use alpha in dense dspr --- .../org/apache/spark/mllib/linalg/distributed/RowMatrix.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index b10857fe7c8a1..07dfadf2f7869 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -440,7 +440,7 @@ object RowMatrix { val n = v.size v match { case dv: DenseVector => - blas.dspr("U", n, 1.0, dv.values, 1, U) + blas.dspr("U", n, alpha, dv.values, 1, U) case sv: SparseVector => val indices = sv.indices val values = sv.values From 9ad096d55a3d8410f04056ebc87dbd8cba391870 Mon Sep 17 00:00:00 2001 From: andrewor14 Date: Wed, 14 May 2014 17:54:53 -0700 Subject: [PATCH 63/82] [Typo] propertes -> properties Author: andrewor14 Closes #780 from andrewor14/submit-typo and squashes the following commits: e70e057 [andrewor14] propertes -> properties --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c6d3cbd2e728b..e86182e4c56ce 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -68,7 +68,7 @@ object SparkSubmit { /** * @return a tuple containing the arguments for the child, a list of classpath - * entries for the child, a list of system propertes, a list of env vars + * entries for the child, a list of system properties, a list of env vars * and the main class for the child */ private[spark] def createLaunchEnv(args: SparkSubmitArguments): (ArrayBuffer[String], From 44165fc91a31e6293a79031c89571e139d2c5356 Mon Sep 17 00:00:00 2001 From: wangfei Date: Wed, 14 May 2014 17:59:11 -0700 Subject: [PATCH 64/82] [SPARK-1826] fix the head notation of package object dsl Author: wangfei Closes #765 from scwf/dslfix and squashes the following commits: d2d1a9d [wangfei] Update package.scala 66ff53b [wangfei] fix the head notation of package object dsl --- .../spark/sql/catalyst/dsl/package.scala | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 78d3a1d8096af..3cf163f9a9a75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -31,26 +31,29 @@ import org.apache.spark.sql.catalyst.types._ * A collection of implicit conversions that create a DSL for constructing catalyst data structures. * * {{{ - * scala> import catalyst.dsl._ + * scala> import org.apache.spark.sql.catalyst.dsl.expressions._ * * // Standard operators are added to expressions. + * scala> import org.apache.spark.sql.catalyst.expressions.Literal * scala> Literal(1) + Literal(1) - * res1: catalyst.expressions.Add = (1 + 1) + * res0: org.apache.spark.sql.catalyst.expressions.Add = (1 + 1) * * // There is a conversion from 'symbols to unresolved attributes. * scala> 'a.attr - * res2: catalyst.analysis.UnresolvedAttribute = 'a + * res1: org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute = 'a * * // These unresolved attributes can be used to create more complicated expressions. * scala> 'a === 'b - * res3: catalyst.expressions.Equals = ('a = 'b) + * res2: org.apache.spark.sql.catalyst.expressions.Equals = ('a = 'b) * * // SQL verbs can be used to construct logical query plans. - * scala> TestRelation('key.int, 'value.string).where('key === 1).select('value).analyze - * res4: catalyst.plans.logical.LogicalPlan = - * Project {value#1} - * Filter (key#0 = 1) - * TestRelation {key#0,value#1} + * scala> import org.apache.spark.sql.catalyst.plans.logical._ + * scala> import org.apache.spark.sql.catalyst.dsl.plans._ + * scala> LocalRelation('key.int, 'value.string).where('key === 1).select('value).analyze + * res3: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = + * Project [value#3] + * Filter (key#2 = 1) + * LocalRelation [key#2,value#3], [] * }}} */ package object dsl { From 2f639957f0bf70dddf1e698aa9e26007fb58bc67 Mon Sep 17 00:00:00 2001 From: Chen Chao Date: Wed, 14 May 2014 18:20:20 -0700 Subject: [PATCH 65/82] default task number misleading in several places private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism){ new HashPartitioner(numPartitions) } it represents that the default task number in Spark Streaming relies on the variable defaultParallelism in SparkContext, which is decided by the config property spark.default.parallelism the property "spark.default.parallelism" refers to https://github.com/apache/spark/pull/389 Author: Chen Chao Closes #766 from CrazyJvm/patch-7 and squashes the following commits: 0b7efba [Chen Chao] Update streaming-programming-guide.md cc5b66c [Chen Chao] default task number misleading in several places --- docs/streaming-programming-guide.md | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 939599aa6855b..0c125eb693a8e 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -522,9 +522,9 @@ common ones are as follows. reduceByKey(func, [numTasks]) When called on a DStream of (K, V) pairs, return a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function. Note: By default, - this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluster) to - do the grouping. You can pass an optional numTasks argument to set a different - number of tasks. + this uses Spark's default number of parallel tasks (2 for local mode, and in cluster mode the number + is determined by the config property spark.default.parallelism) to do the grouping. + You can pass an optional numTasks argument to set a different number of tasks. join(otherStream, [numTasks]) @@ -743,8 +743,9 @@ said two parameters - windowLength and slideInterval. When called on a DStream of (K, V) pairs, returns a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function func over batches in a sliding window. Note: By default, this uses Spark's default number of - parallel tasks (2 for local machine, 8 for a cluster) to do the grouping. You can pass an optional - numTasks argument to set a different number of tasks. + parallel tasks (2 for local mode, and in cluster mode the number is determined by the config + property spark.default.parallelism) to do the grouping. You can pass an optional + numTasks argument to set a different number of tasks. @@ -956,9 +957,10 @@ before further processing. ### Level of Parallelism in Data Processing Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` -and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of -parallelism as an argument (see the -[`PairDStreamFunctions`](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) +and `reduceByKeyAndWindow`, the default number of parallel tasks is decided by the [config property] +(configuration.html#spark-properties) `spark.default.parallelism`. You can pass the level of +parallelism as an argument (see [`PairDStreamFunctions`] +(api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) documentation), or set the [config property](configuration.html#spark-properties) `spark.default.parallelism` to change the default. From ad4e60ee7e2c49c24a9972312915f7f7253c7679 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 14 May 2014 21:13:41 -0700 Subject: [PATCH 66/82] [SPARK-1840] SparkListenerBus prints out scary error message when terminated normally Running SparkPi example gave this error. ``` Pi is roughly 3.14374 14/05/14 18:16:19 ERROR Utils: Uncaught exception in thread SparkListenerBus scala.runtime.NonLocalReturnControl$mcV$sp ``` This is due to the catch-all in the SparkListenerBus, which logged control throwable used by scala system Author: Tathagata Das Closes #783 from tdas/controlexception-fix and squashes the following commits: a466c8d [Tathagata Das] Ignored control exceptions when logging all exceptions. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d041bfa66a1cd..388f7222428db 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1149,6 +1149,8 @@ private[spark] object Utils extends Logging { try { f } catch { + case ct: ControlThrowable => + throw ct case t: Throwable => logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) throw t From f10de042b8e86adf51b70bae2d8589a5cbf02935 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 14 May 2014 21:45:20 -0700 Subject: [PATCH 67/82] Add language tabs and Python version to interactive part of quick-start This is an addition of some stuff that was missed in https://issues.apache.org/jira/browse/SPARK-1567. I've also updated the doc to show submitting the Python application with spark-submit. Author: Matei Zaharia Closes #782 from mateiz/spark-1567-extra and squashes the following commits: 6f8f2aa [Matei Zaharia] tweaks 9ed9874 [Matei Zaharia] tweaks ae67c3e [Matei Zaharia] tweak b303ba3 [Matei Zaharia] tweak 1433a4d [Matei Zaharia] Add language tabs and Python version to interactive part of quick-start guide --- docs/python-programming-guide.md | 2 +- docs/quick-start.md | 151 +++++++++++++++++++++++++++---- 2 files changed, 133 insertions(+), 20 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 2ce2c346d7b04..17675acba6bb8 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -121,7 +121,7 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env # Standalone Programs PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/spark-submit`. -The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application. +The Quick Start guide includes a [complete example](quick-start.html#standalone-applications) of a standalone Python application. Code dependencies can be deployed by passing .zip or .egg files in the `--py-files` option of `spark-submit`: diff --git a/docs/quick-start.md b/docs/quick-start.md index a4d01487bb494..33a0df1036424 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -6,7 +6,9 @@ title: Quick Start * This will become a table of contents (this text will be scraped). {:toc} -This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will not need much for this), then show how to write standalone applications in Scala, Java, and Python. +This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's +interactive shell (in Python or Scala), +then show how to write standalone applications in Java, Scala, and Python. See the [programming guide](scala-programming-guide.html) for a more complete reference. To follow along with this guide, first download a packaged release of Spark from the @@ -17,8 +19,12 @@ you can download a package for any version of Hadoop. ## Basics -Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively. -Start the shell by running the following in the Spark directory. +Spark's shell provides a simple way to learn the API, as well as a powerful tool to analyze data interactively. +It is available in either Scala (which runs on the Java VM and is thus a good way to use existing Java libraries) +or Python. Start it by running the following in the Spark directory: + +
+
./bin/spark-shell @@ -33,7 +39,7 @@ RDDs have _[actions](scala-programming-guide.html#actions)_, which return values {% highlight scala %} scala> textFile.count() // Number of items in this RDD -res0: Long = 74 +res0: Long = 126 scala> textFile.first() // First item in this RDD res1: String = # Apache Spark @@ -53,12 +59,53 @@ scala> textFile.filter(line => line.contains("Spark")).count() // How many lines res3: Long = 15 {% endhighlight %} +
+
+ + ./bin/pyspark + +Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory: + +{% highlight python %} +>>> textFile = sc.textFile("README.md") +{% endhighlight %} + +RDDs have _[actions](scala-programming-guide.html#actions)_, which return values, and _[transformations](scala-programming-guide.html#transformations)_, which return pointers to new RDDs. Let's start with a few actions: + +{% highlight python %} +>>> textFile.count() # Number of items in this RDD +126 + +>>> textFile.first() # First item in this RDD +u'# Apache Spark' +{% endhighlight %} + +Now let's use a transformation. We will use the [`filter`](scala-programming-guide.html#transformations) transformation to return a new RDD with a subset of the items in the file. + +{% highlight python %} +>>> linesWithSpark = textFile.filter(lambda line: "Spark" in line) +{% endhighlight %} + +We can chain together transformations and actions: + +{% highlight python %} +>>> textFile.filter(lambda line: "Spark" in line).count() # How many lines contain "Spark"? +15 +{% endhighlight %} + +
+
+ + ## More on RDD Operations RDD actions and transformations can be used for more complex computations. Let's say we want to find the line with the most words: +
+
+ {% highlight scala %} scala> textFile.map(line => line.split(" ").size).reduce((a, b) => if (a > b) a else b) -res4: Long = 16 +res4: Long = 15 {% endhighlight %} This first maps a line to an integer value, creating a new RDD. `reduce` is called on that RDD to find the largest line count. The arguments to `map` and `reduce` are Scala function literals (closures), and can use any language feature or Scala/Java library. For example, we can easily call functions declared elsewhere. We'll use `Math.max()` function to make this code easier to understand: @@ -68,26 +115,69 @@ scala> import java.lang.Math import java.lang.Math scala> textFile.map(line => line.split(" ").size).reduce((a, b) => Math.max(a, b)) -res5: Int = 16 +res5: Int = 15 {% endhighlight %} One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can implement MapReduce flows easily: {% highlight scala %} scala> val wordCounts = textFile.flatMap(line => line.split(" ")).map(word => (word, 1)).reduceByKey((a, b) => a + b) -wordCounts: spark.RDD[(java.lang.String, Int)] = spark.ShuffledAggregatedRDD@71f027b8 +wordCounts: spark.RDD[(String, Int)] = spark.ShuffledAggregatedRDD@71f027b8 {% endhighlight %} Here, we combined the [`flatMap`](scala-programming-guide.html#transformations), [`map`](scala-programming-guide.html#transformations) and [`reduceByKey`](scala-programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (String, Int) pairs. To collect the word counts in our shell, we can use the [`collect`](scala-programming-guide.html#actions) action: {% highlight scala %} scala> wordCounts.collect() -res6: Array[(java.lang.String, Int)] = Array((need,2), ("",43), (Extra,3), (using,1), (passed,1), (etc.,1), (its,1), (`/usr/local/lib/libmesos.so`,1), (`SCALA_HOME`,1), (option,1), (these,1), (#,1), (`PATH`,,2), (200,1), (To,3),... +res6: Array[(String, Int)] = Array((means,1), (under,2), (this,3), (Because,1), (Python,2), (agree,1), (cluster.,1), ...) {% endhighlight %} +
+
+ +{% highlight python %} +>>> textFile.map(lambda line: len(line.split())).reduce(lambda a, b: a if (a > b) else b) +15 +{% endhighlight %} + +This first maps a line to an integer value, creating a new RDD. `reduce` is called on that RDD to find the largest line count. The arguments to `map` and `reduce` are Python [anonymous functions (lambdas)](https://docs.python.org/2/reference/expressions.html#lambda), +but we can also pass any top-level Python function we want. +For example, we'll define a `max` function to make this code easier to understand: + +{% highlight python %} +>>> def max(a, b): +... if a > b: +... return a +... else: +... return b +... + +>>> textFile.map(lambda line: len(line.split())).reduce(max) +15 +{% endhighlight %} + +One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can implement MapReduce flows easily: + +{% highlight python %} +>>> wordCounts = textFile.flatMap(lambda line: line.split()).map(lambda word: (word, 1)).reduceByKey(lambda a, b: a+b) +{% endhighlight %} + +Here, we combined the [`flatMap`](scala-programming-guide.html#transformations), [`map`](scala-programming-guide.html#transformations) and [`reduceByKey`](scala-programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (string, int) pairs. To collect the word counts in our shell, we can use the [`collect`](scala-programming-guide.html#actions) action: + +{% highlight python %} +>>> wordCounts.collect() +[(u'and', 9), (u'A', 1), (u'webpage', 1), (u'README', 1), (u'Note', 1), (u'"local"', 1), (u'variable', 1), ...] +{% endhighlight %} + +
+
+ ## Caching Spark also supports pulling data sets into a cluster-wide in-memory cache. This is very useful when data is accessed repeatedly, such as when querying a small "hot" dataset or when running an iterative algorithm like PageRank. As a simple example, let's mark our `linesWithSpark` dataset to be cached: +
+
+ {% highlight scala %} scala> linesWithSpark.cache() res7: spark.RDD[String] = spark.FilteredRDD@17e51082 @@ -99,12 +189,33 @@ scala> linesWithSpark.count() res9: Long = 15 {% endhighlight %} -It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is +It may seem silly to use Spark to explore and cache a 100-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). -# A Standalone Application +
+
+ +{% highlight python %} +>>> linesWithSpark.cache() + +>>> linesWithSpark.count() +15 + +>>> linesWithSpark.count() +15 +{% endhighlight %} + +It may seem silly to use Spark to explore and cache a 100-line text file. The interesting part is +that these same functions can be used on very large data sets, even when they are striped across +tens or hundreds of nodes. You can also do this interactively by connecting `bin/pyspark` to +a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). + +
+
+ +# Standalone Applications Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. @@ -115,7 +226,7 @@ We'll create a very simple Spark application in Scala. So simple, in fact, that named `SimpleApp.scala`: {% highlight scala %} -/*** SimpleApp.scala ***/ +/* SimpleApp.scala */ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import org.apache.spark.SparkConf @@ -194,7 +305,7 @@ This example will use Maven to compile an application jar, but any similar build We'll create a very simple Spark application, `SimpleApp.java`: {% highlight java %} -/*** SimpleApp.java ***/ +/* SimpleApp.java */ import org.apache.spark.api.java.*; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.Function; @@ -309,16 +420,18 @@ Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark i As with the Scala and Java examples, we use a SparkContext to create RDDs. We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference. -For applications that use custom classes or third-party libraries, we can add those code -dependencies to SparkContext to ensure that they will be available on remote machines; this is -described in more detail in the [Python programming guide](python-programming-guide.html). +For applications that use custom classes or third-party libraries, we can also add code +dependencies to `spark-submit` through its `--py-files` argument by packaging them into a +.zip file (see `spark-submit --help` for details). `SimpleApp` is simple enough that we do not need to specify any code dependencies. -We can run this application using the `bin/pyspark` script: +We can run this application using the `bin/spark-submit` script: {% highlight python %} -$ cd $SPARK_HOME -$ ./bin/pyspark SimpleApp.py +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit \ + --master local[4] \ + SimpleApp.py ... Lines with a: 46, Lines with b: 23 {% endhighlight python %} @@ -326,7 +439,7 @@ Lines with a: 46, Lines with b: 23 -# Where to go from here +# Where to Go from Here Congratulations on running your first Spark application! * For an in-depth overview of the API see "Programming Guides" menu section. From 21570b463388194877003318317aafd842800cac Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 14 May 2014 22:24:04 -0700 Subject: [PATCH 68/82] Documentation: Encourage use of reduceByKey instead of groupByKey. Author: Patrick Wendell Closes #784 from pwendell/group-by-key and squashes the following commits: 9b4505f [Patrick Wendell] Small fix 6347924 [Patrick Wendell] Documentation: Encourage use of reduceByKey instead of groupByKey. --- .../org/apache/spark/api/java/JavaPairRDD.scala | 12 ++++++++++++ .../org/apache/spark/rdd/PairRDDFunctions.scala | 12 ++++++++++++ docs/scala-programming-guide.md | 4 ++++ python/pyspark/rdd.py | 4 ++++ 4 files changed, 32 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 554c065358648..4c8f9ed6fbc02 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -263,6 +263,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * will provide much better performance. */ def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(partitioner))) @@ -270,6 +274,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * will provide much better performance. */ def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) @@ -380,6 +388,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[JavaPairRDD.reduceByKey]] or [[JavaPairRDD.combineByKey]] + * will provide much better performance. */ def groupByKey(): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey())) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index bc6d204434ad8..223fef79261d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -264,6 +264,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] + * will provide much better performance. */ def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // groupByKey shouldn't use map side combine because map side combine does not @@ -280,6 +284,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] + * will provide much better performance. */ def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) @@ -365,6 +373,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. + * + * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over + * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] + * will provide much better performance, */ def groupByKey(): RDD[(K, Iterable[V])] = { groupByKey(defaultPartitioner(self)) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 3ed86e460c01c..edaa7d0639a18 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -196,6 +196,10 @@ The following tables list the transformations and actions currently supported (s groupByKey([numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, Seq[V]) pairs.
+Note: If you are grouping in order to perform an aggregation (such as a sum or + average) over each key, using `reduceByKey` or `combineByKey` will yield much better + performance. +
Note: By default, if the RDD already has a partitioner, the task number is decided by the partition number of the partitioner, or else relies on the value of spark.default.parallelism if the property is set , otherwise depends on the partition number of the RDD. You can pass an optional numTasks argument to set a different number of tasks. diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 4f74824ba4cf2..07578b8d937fc 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1152,6 +1152,10 @@ def groupByKey(self, numPartitions=None): Group the values for each key in the RDD into a single sequence. Hash-partitions the resulting RDD with into numPartitions partitions. + Note: If you are grouping in order to perform an aggregation (such as a + sum or average) over each key, using reduceByKey will provide much better + performance. + >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> map((lambda (x,y): (x, list(y))), sorted(x.groupByKey().collect())) [('a', [1, 1]), ('b', [1])] From 46324279dae2fa803267d788f7c56b0ed643b4c8 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 14 May 2014 22:24:41 -0700 Subject: [PATCH 69/82] Package docs This is a few changes based on the original patch by @scrapcodes. Author: Prashant Sharma Author: Patrick Wendell Closes #785 from pwendell/package-docs and squashes the following commits: c32b731 [Patrick Wendell] Changes based on Prashant's patch c0463d3 [Prashant Sharma] added eof new line ce8bf73 [Prashant Sharma] Added eof new line to all files. 4c35f2e [Prashant Sharma] SPARK-1563 Add package-info.java and package.scala files for all packages that appear in docs --- .../org/apache/spark/bagel/package-info.java | 21 ++++++++++++++ .../org/apache/spark/bagel/package.scala | 23 +++++++++++++++ .../spark/api/java/function/package-info.java | 23 +++++++++++++++ .../spark/api/java/function/package.scala | 25 +++++++++++++++++ .../apache/spark/annotation/package-info.java | 23 +++++++++++++++ .../org/apache/spark/annotation/package.scala | 25 +++++++++++++++++ .../apache/spark/api/java/package-info.java | 21 ++++++++++++++ .../apache/spark/broadcast/package-info.java | 21 ++++++++++++++ .../org/apache/spark/broadcast/package.scala | 2 +- .../apache/spark/executor/package-info.java | 21 ++++++++++++++ .../org/apache/spark/executor/package.scala | 24 ++++++++++++++++ .../org/apache/spark/io/package-info.java | 21 ++++++++++++++ .../scala/org/apache/spark/io/package.scala | 23 +++++++++++++++ .../apache/spark/metrics/sink/package.scala | 23 +++++++++++++++ .../apache/spark/metrics/source/package.scala | 23 +++++++++++++++ .../org/apache/spark/partial/package.scala | 28 +++++++++++++++++++ .../org/apache/spark/rdd/package-info.java | 21 ++++++++++++++ .../scala/org/apache/spark/rdd/package.scala | 23 +++++++++++++++ .../apache/spark/scheduler/package-info.java | 21 ++++++++++++++ .../org/apache/spark/scheduler/package.scala | 24 ++++++++++++++++ .../apache/spark/serializer/package-info.java | 21 ++++++++++++++ .../org/apache/spark/serializer/package.scala | 25 +++++++++++++++++ .../org/apache/spark/util/package-info.java | 21 ++++++++++++++ .../scala/org/apache/spark/util/package.scala | 23 +++++++++++++++ .../spark/util/random/package-info.java | 21 ++++++++++++++ .../apache/spark/util/random/package.scala | 23 +++++++++++++++ .../spark/streaming/flume/package-info.java | 21 ++++++++++++++ .../spark/streaming/flume/package.scala | 23 +++++++++++++++ .../spark/streaming/kafka/package-info.java | 21 ++++++++++++++ .../spark/streaming/kafka/package.scala | 23 +++++++++++++++ .../spark/streaming/mqtt/package-info.java | 21 ++++++++++++++ .../apache/spark/streaming/mqtt/package.scala | 23 +++++++++++++++ .../spark/streaming/twitter/package-info.java | 21 ++++++++++++++ .../spark/streaming/twitter/package.scala | 23 +++++++++++++++ .../spark/streaming/zeromq/package-info.java | 21 ++++++++++++++ .../spark/streaming/zeromq/package.scala | 23 +++++++++++++++ .../apache/spark/graphx/lib/package-info.java | 21 ++++++++++++++ .../org/apache/spark/graphx/lib/package.scala | 23 +++++++++++++++ .../org/apache/spark/graphx/package-info.java | 22 +++++++++++++++ .../spark/graphx/util/package-info.java | 21 ++++++++++++++ .../apache/spark/graphx/util/package.scala | 23 +++++++++++++++ .../spark/mllib/api/python/package.scala | 25 +++++++++++++++++ .../org/apache/spark/mllib/package-info.java | 21 ++++++++++++++ .../org/apache/spark/mllib/package.scala | 23 +++++++++++++++ .../org/apache/spark/sql/package-info.java | 21 ++++++++++++++ .../apache/spark/sql/hive/package-info.java | 18 ++++++++++++ .../org/apache/spark/sql/hive/package.scala | 20 +++++++++++++ .../streaming/api/java/package-info.java | 21 ++++++++++++++ .../spark/streaming/api/java/package.scala | 23 +++++++++++++++ .../spark/streaming/dstream/package-info.java | 21 ++++++++++++++ .../spark/streaming/dstream/package.scala | 24 ++++++++++++++++ 51 files changed, 1116 insertions(+), 1 deletion(-) create mode 100644 bagel/src/main/scala/org/apache/spark/bagel/package-info.java create mode 100644 bagel/src/main/scala/org/apache/spark/bagel/package.scala create mode 100644 core/src/main/java/org/apache/spark/api/java/function/package-info.java create mode 100644 core/src/main/java/org/apache/spark/api/java/function/package.scala create mode 100644 core/src/main/scala/org/apache/spark/annotation/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/annotation/package.scala create mode 100644 core/src/main/scala/org/apache/spark/api/java/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/broadcast/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/executor/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/executor/package.scala create mode 100644 core/src/main/scala/org/apache/spark/io/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/io/package.scala create mode 100644 core/src/main/scala/org/apache/spark/metrics/sink/package.scala create mode 100644 core/src/main/scala/org/apache/spark/metrics/source/package.scala create mode 100644 core/src/main/scala/org/apache/spark/partial/package.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/rdd/package.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/scheduler/package.scala create mode 100644 core/src/main/scala/org/apache/spark/serializer/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/serializer/package.scala create mode 100644 core/src/main/scala/org/apache/spark/util/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/util/package.scala create mode 100644 core/src/main/scala/org/apache/spark/util/random/package-info.java create mode 100644 core/src/main/scala/org/apache/spark/util/random/package.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala create mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java create mode 100644 external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java create mode 100644 external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java create mode 100644 external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/package-info.java create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/util/package.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/package-info.java create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/package-info.java create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/package.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/package.scala diff --git a/bagel/src/main/scala/org/apache/spark/bagel/package-info.java b/bagel/src/main/scala/org/apache/spark/bagel/package-info.java new file mode 100644 index 0000000000000..81f26f276549f --- /dev/null +++ b/bagel/src/main/scala/org/apache/spark/bagel/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Bagel: An implementation of Pregel in Spark. THIS IS DEPRECATED - use Spark's GraphX library. + */ +package org.apache.spark.bagel; \ No newline at end of file diff --git a/bagel/src/main/scala/org/apache/spark/bagel/package.scala b/bagel/src/main/scala/org/apache/spark/bagel/package.scala new file mode 100644 index 0000000000000..2fb1934579781 --- /dev/null +++ b/bagel/src/main/scala/org/apache/spark/bagel/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Bagel: An implementation of Pregel in Spark. THIS IS DEPRECATED - use Spark's GraphX library. + */ +package object bagel diff --git a/core/src/main/java/org/apache/spark/api/java/function/package-info.java b/core/src/main/java/org/apache/spark/api/java/function/package-info.java new file mode 100644 index 0000000000000..463a42f23342c --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Set of interfaces to represent functions in Spark's Java API. Users create implementations of + * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's + * Java programming guide for more details. + */ +package org.apache.spark.api.java.function; \ No newline at end of file diff --git a/core/src/main/java/org/apache/spark/api/java/function/package.scala b/core/src/main/java/org/apache/spark/api/java/function/package.scala new file mode 100644 index 0000000000000..7f91de653a64a --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/package.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java + +/** + * Set of interfaces to represent functions in Spark's Java API. Users create implementations of + * these interfaces to pass functions to various Java API methods for Spark. Please visit Spark's + * Java programming guide for more details. + */ +package object function \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/annotation/package-info.java b/core/src/main/scala/org/apache/spark/annotation/package-info.java new file mode 100644 index 0000000000000..12c7afe6f108a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/package-info.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark annotations to mark an API experimental or intended only for advanced usages by developers. + * This package consist of these annotations, which are used project wide and are reflected in + * Scala and Java docs. + */ +package org.apache.spark.annotation; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/annotation/package.scala b/core/src/main/scala/org/apache/spark/annotation/package.scala new file mode 100644 index 0000000000000..c3f4026a29d44 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/package.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark annotations to mark an API experimental or intended only for advanced usages by developers. + * This package consist of these annotations, which are used project wide and are reflected in + * Scala and Java docs. + */ +package object annotation diff --git a/core/src/main/scala/org/apache/spark/api/java/package-info.java b/core/src/main/scala/org/apache/spark/api/java/package-info.java new file mode 100644 index 0000000000000..10a480fc78e44 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/java/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark Java programming APIs. + */ +package org.apache.spark.api.java; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/broadcast/package-info.java b/core/src/main/scala/org/apache/spark/broadcast/package-info.java new file mode 100644 index 0000000000000..1510e6e84c7a8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark's broadcast variables, used to broadcast immutable datasets to all nodes. + */ +package org.apache.spark.broadcast; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/broadcast/package.scala b/core/src/main/scala/org/apache/spark/broadcast/package.scala index 01bf88629a7dd..4f2c3c07161a3 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/package.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/package.scala @@ -18,7 +18,7 @@ package org.apache.spark /** - * Package for broadcast variables. See [[broadcast.Broadcast]] for details. + * Spark's broadcast variables, used to broadcast immutable datasets to all nodes. */ package object broadcast { // For package docs only diff --git a/core/src/main/scala/org/apache/spark/executor/package-info.java b/core/src/main/scala/org/apache/spark/executor/package-info.java new file mode 100644 index 0000000000000..dd3b6815fb45f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Package for executor components used with various cluster managers. + */ +package org.apache.spark.executor; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/executor/package.scala b/core/src/main/scala/org/apache/spark/executor/package.scala new file mode 100644 index 0000000000000..ef471d761d4b3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Executor components used with various cluster managers. + * See [[org.apache.spark.executor.Executor]]. + */ +package object executor diff --git a/core/src/main/scala/org/apache/spark/io/package-info.java b/core/src/main/scala/org/apache/spark/io/package-info.java new file mode 100644 index 0000000000000..bea1bfdb63751 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/io/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * IO codecs used for compression. + */ +package org.apache.spark.io; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/io/package.scala b/core/src/main/scala/org/apache/spark/io/package.scala new file mode 100644 index 0000000000000..f987e66a5dc44 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/io/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * IO codecs used for compression. See [[org.apache.spark.io.CompressionCodec]]. + */ +package object io diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/package.scala b/core/src/main/scala/org/apache/spark/metrics/sink/package.scala new file mode 100644 index 0000000000000..90e3aa70b99ef --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/sink/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics + +/** + * Sinks used in Spark's metrics system. + */ +package object sink diff --git a/core/src/main/scala/org/apache/spark/metrics/source/package.scala b/core/src/main/scala/org/apache/spark/metrics/source/package.scala new file mode 100644 index 0000000000000..1e59fed6f7d7a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/source/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics + +/** + * Sources used in Spark's metrics system. + */ +package object source diff --git a/core/src/main/scala/org/apache/spark/partial/package.scala b/core/src/main/scala/org/apache/spark/partial/package.scala new file mode 100644 index 0000000000000..62dc5cd25a164 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/partial/package.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * :: Experimental :: + * + * Support for approximate results. This provides convenient api and also implementation for + * approximate calculation. + * + * @see [[org.apache.spark.rdd.RDD.countApprox]] + */ +package object partial diff --git a/core/src/main/scala/org/apache/spark/rdd/package-info.java b/core/src/main/scala/org/apache/spark/rdd/package-info.java new file mode 100644 index 0000000000000..176cc58179fb0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Provides implementation's of various RDDs. + */ +package org.apache.spark.rdd; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/rdd/package.scala b/core/src/main/scala/org/apache/spark/rdd/package.scala new file mode 100644 index 0000000000000..55fc6e4d2b4df --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Provides several RDD implementations. See [[org.apache.spark.rdd.RDD]]. + */ +package object rdd diff --git a/core/src/main/scala/org/apache/spark/scheduler/package-info.java b/core/src/main/scala/org/apache/spark/scheduler/package-info.java new file mode 100644 index 0000000000000..5b4a628d3cee4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark's DAG scheduler. + */ +package org.apache.spark.scheduler; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/scheduler/package.scala b/core/src/main/scala/org/apache/spark/scheduler/package.scala new file mode 100644 index 0000000000000..f0dbfc2ac5f48 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark's scheduling components. This includes the [[org.apache.spark.scheduler.DAGScheduler]] and + * lower level [[org.apache.spark.scheduler.TaskScheduler]]. + */ +package object scheduler diff --git a/core/src/main/scala/org/apache/spark/serializer/package-info.java b/core/src/main/scala/org/apache/spark/serializer/package-info.java new file mode 100644 index 0000000000000..4c0b73ab36a00 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/serializer/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Pluggable serializers for RDD and shuffle data. + */ +package org.apache.spark.serializer; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/serializer/package.scala b/core/src/main/scala/org/apache/spark/serializer/package.scala new file mode 100644 index 0000000000000..8f09ef6f5b34c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/serializer/package.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Pluggable serializers for RDD and shuffle data. + * + * @see [[org.apache.spark.serializer.Serializer]] + */ +package object serializer diff --git a/core/src/main/scala/org/apache/spark/util/package-info.java b/core/src/main/scala/org/apache/spark/util/package-info.java new file mode 100644 index 0000000000000..819f54ee41a79 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark utilities. + */ +package org.apache.spark.util; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/util/package.scala b/core/src/main/scala/org/apache/spark/util/package.scala new file mode 100644 index 0000000000000..c7099b85d225f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark utilities. + */ +package object util diff --git a/core/src/main/scala/org/apache/spark/util/random/package-info.java b/core/src/main/scala/org/apache/spark/util/random/package-info.java new file mode 100644 index 0000000000000..62c3762dd11b6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Utilities for random number generation. + */ +package org.apache.spark.util.random; \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/util/random/package.scala b/core/src/main/scala/org/apache/spark/util/random/package.scala new file mode 100644 index 0000000000000..f361125fe2e8a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +/** + * Utilities for random number generation. + */ +package object random diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java new file mode 100644 index 0000000000000..d31aa5f5c096c --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark streaming receiver for Flume. + */ +package org.apache.spark.streaming.flume; \ No newline at end of file diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala new file mode 100644 index 0000000000000..9bfab68c4b8b7 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Spark streaming receiver for Flume. + */ +package object flume diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java new file mode 100644 index 0000000000000..947bae115a620 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Kafka receiver for spark streaming. + */ +package org.apache.spark.streaming.kafka; \ No newline at end of file diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala new file mode 100644 index 0000000000000..47c5187f8751f --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Kafka receiver for spark streaming, + */ +package object kafka diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java new file mode 100644 index 0000000000000..728e0d8663d01 --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * MQTT receiver for Spark Streaming. + */ +package org.apache.spark.streaming.mqtt; \ No newline at end of file diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala new file mode 100644 index 0000000000000..63d0d138183a9 --- /dev/null +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * MQTT receiver for Spark Streaming. + */ +package object mqtt diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java new file mode 100644 index 0000000000000..258c0950a0aa7 --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Twitter feed receiver for spark streaming. + */ +package org.apache.spark.streaming.twitter; \ No newline at end of file diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala new file mode 100644 index 0000000000000..580e37fa8f814 --- /dev/null +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Twitter feed receiver for spark streaming. + */ +package object twitter diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java new file mode 100644 index 0000000000000..587c524e2120f --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Zeromq receiver for spark streaming. + */ +package org.apache.spark.streaming.zeromq; \ No newline at end of file diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala new file mode 100644 index 0000000000000..65e6e57f2c05d --- /dev/null +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Zeromq receiver for spark streaming. + */ +package object zeromq diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java new file mode 100644 index 0000000000000..49ad7263141ce --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Internal support for MLLib Python API. + */ +package org.apache.spark.graphx.lib; \ No newline at end of file diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala new file mode 100644 index 0000000000000..7cfd617a32a34 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx + +/** + * Various analytics functions for graphs. + */ +package object lib diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/package-info.java new file mode 100644 index 0000000000000..f659cc518ebd3 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * ALPHA COMPONENT + * GraphX is a graph processing framework built on top of Spark. + */ +package org.apache.spark.graphx; \ No newline at end of file diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java new file mode 100644 index 0000000000000..90cd1d46db177 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Collections of utilities used by graphx. + */ +package org.apache.spark.graphx.util; \ No newline at end of file diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/package.scala new file mode 100644 index 0000000000000..2303541d2e427 --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx + +/** + * Collections of utilities used by graphx. + */ +package object util diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala new file mode 100644 index 0000000000000..87bdc8558aaf5 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.api + +/** + * Internal support for MLLib Python API. + * + * @see [[org.apache.spark.mllib.api.python.PythonMLLibAPI]] + */ +package object python diff --git a/mllib/src/main/scala/org/apache/spark/mllib/package-info.java b/mllib/src/main/scala/org/apache/spark/mllib/package-info.java new file mode 100644 index 0000000000000..4991bc9e972c0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Spark's machine learning library. + */ +package org.apache.spark.mllib; \ No newline at end of file diff --git a/mllib/src/main/scala/org/apache/spark/mllib/package.scala b/mllib/src/main/scala/org/apache/spark/mllib/package.scala new file mode 100644 index 0000000000000..5c2b2160c030e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * Spark's machine learning library. + */ +package object mllib diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java new file mode 100644 index 0000000000000..53603614518f5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Allows the execution of relational queries, including those expressed in SQL using Spark. + */ +package org.apache.spark.sql; \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package-info.java b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package-info.java new file mode 100644 index 0000000000000..8b29fa7d1a8f7 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package-info.java @@ -0,0 +1,18 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive; \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala new file mode 100644 index 0000000000000..a6c8ed4f7e866 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +package object hive diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java new file mode 100644 index 0000000000000..d43d949d76bb3 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Java APIs for spark streaming. + */ +package org.apache.spark.streaming.api.java; \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/package.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package.scala new file mode 100644 index 0000000000000..2479670f2a5bb --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.api + +/** + * Spark streaming's Java API. + */ +package object java diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java new file mode 100644 index 0000000000000..05ca2ddffd3c0 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Various implementations of DStreams. + */ +package org.apache.spark.streaming.dstream; \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/package.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package.scala new file mode 100644 index 0000000000000..0179b92503e5d --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +/** + * Various implementations of DStream's. + * @see [[org.apache.spark.streaming.dstream.DStream]] + */ +package object dstream From bae07e36a6e0fb7982405316646b452b4ff06acc Mon Sep 17 00:00:00 2001 From: witgo Date: Wed, 14 May 2014 22:26:26 -0700 Subject: [PATCH 70/82] fix different versions of commons-lang dependency and apache/spark#746 addendum Author: witgo Closes #754 from witgo/commons-lang and squashes the following commits: 3ebab31 [witgo] merge master f3b8fa2 [witgo] merge master 2083fae [witgo] repeat definition 5599cdb [witgo] multiple version of sbt dependency c1b66a1 [witgo] fix different versions of commons-lang dependency --- project/SparkBuild.scala | 12 +++++++----- sql/hive/pom.xml | 8 ++++++++ 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6adec556322ba..29dcd8678b476 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -318,6 +318,8 @@ object SparkBuild extends Build { val excludeFastutil = ExclusionRule(organization = "it.unimi.dsi") val excludeJruby = ExclusionRule(organization = "org.jruby") val excludeThrift = ExclusionRule(organization = "org.apache.thrift") + val excludeCommonsLang = ExclusionRule(organization = "commons-lang") + val excludeServletApi = ExclusionRule(organization = "javax.servlet", artifact = "servlet-api") def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { @@ -491,7 +493,7 @@ object SparkBuild extends Build { javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( "org.spark-project.hive" % "hive-metastore" % hiveVersion, - "org.spark-project.hive" % "hive-exec" % hiveVersion, + "org.spark-project.hive" % "hive-exec" % hiveVersion excludeAll(excludeCommonsLang, excludeCommonsLogging), "org.spark-project.hive" % "hive-serde" % hiveVersion ), // Multiple queries rely on the TestHive singleton. See comments there for more details. @@ -564,10 +566,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( // Exclude rule required for all ? "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm) + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), + "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeServletApi) ) ) diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 9254b70e64a08..4d0b2fa1452a2 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -57,6 +57,10 @@ commons-logging commons-logging + + commons-lang + commons-lang +
@@ -76,6 +80,10 @@ commons-logging commons-logging-api + + commons-lang + commons-lang + From 514157f2e4c341378f4a606b66562f39fe3eba5e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 14 May 2014 23:48:03 -0700 Subject: [PATCH 71/82] HOTFIX: Don't build Javadoc in Maven when creating releases. Because we've added java package descriptions in some packages that don't have any Java files, running the Javadoc target hits this issue: http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4492654 To fix this I've simply removed the javadoc target when publishing releases. --- dev/create-release/create-release.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index c4e74990860b6..c96afe822c6a9 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -49,16 +49,18 @@ if [[ ! "$@" =~ --package-only ]]; then mvn -Pyarn release:clean mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ + -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare mvn -DskipTests \ - -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ + -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ + -Dmaven.javadoc.skip=true \ -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform From 3abe2b734a5578966f671c34f1de34b4446b90f1 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 15 May 2014 11:05:39 -0700 Subject: [PATCH 72/82] SPARK-1846 Ignore logs directory in RAT checks https://issues.apache.org/jira/browse/SPARK-1846 Author: Andrew Ash Closes #793 from ash211/SPARK-1846 and squashes the following commits: 3f50db5 [Andrew Ash] SPARK-1846 Ignore logs directory in RAT checks --- .rat-excludes | 1 + 1 file changed, 1 insertion(+) diff --git a/.rat-excludes b/.rat-excludes index 689467847d80d..15589702c5599 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -48,3 +48,4 @@ dist/* .*out .*ipr .*iws +logs From db8cc6f28abe4326cea6f53feb604920e4867a27 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 15 May 2014 11:20:21 -0700 Subject: [PATCH 73/82] [SPARK-1845] [SQL] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of ... ...Scala collections. When I execute `orderBy` or `limit` for `SchemaRDD` including `ArrayType` or `MapType`, `SparkSqlSerializer` throws the following exception: ``` com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.$colon$colon ``` or ``` com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.Vector ``` or ``` com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.HashMap$HashTrieMap ``` and so on. This is because registrations of serializers for each concrete collections are missing in `SparkSqlSerializer`. I believe it should use `AllScalaRegistrar`. `AllScalaRegistrar` covers a lot of serializers for concrete classes of `Seq`, `Map` for `ArrayType`, `MapType`. Author: Takuya UESHIN Closes #790 from ueshin/issues/SPARK-1845 and squashes the following commits: d1ed992 [Takuya UESHIN] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of Scala collections. --- .../sql/execution/SparkSqlSerializer.scala | 28 ++--------------- .../org/apache/spark/sql/DslQuerySuite.scala | 24 +++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 30 +++++++++++++++++++ .../scala/org/apache/spark/sql/TestData.scala | 10 +++++++ 4 files changed, 66 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 94c2a249ef8f8..34b355e906695 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} +import com.twitter.chill.AllScalaRegistrar import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.KryoSerializer @@ -35,22 +36,14 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co val kryo = new Kryo() kryo.setRegistrationRequired(false) kryo.register(classOf[MutablePair[_, _]]) - kryo.register(classOf[Array[Any]]) - // This is kinda hacky... - kryo.register(classOf[scala.collection.immutable.Map$Map1], new MapSerializer) - kryo.register(classOf[scala.collection.immutable.Map$Map2], new MapSerializer) - kryo.register(classOf[scala.collection.immutable.Map$Map3], new MapSerializer) - kryo.register(classOf[scala.collection.immutable.Map$Map4], new MapSerializer) - kryo.register(classOf[scala.collection.immutable.Map[_,_]], new MapSerializer) - kryo.register(classOf[scala.collection.Map[_,_]], new MapSerializer) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow]) kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow]) kryo.register(classOf[com.clearspring.analytics.stream.cardinality.HyperLogLog], new HyperLogLogSerializer) - kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]]) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) kryo.setReferences(false) kryo.setClassLoader(Utils.getSparkClassLoader) + new AllScalaRegistrar().apply(kryo) kryo } } @@ -97,20 +90,3 @@ private[sql] class HyperLogLogSerializer extends Serializer[HyperLogLog] { HyperLogLog.Builder.build(bytes) } } - -/** - * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize - * them as `Array[(k,v)]`. - */ -private[sql] class MapSerializer extends Serializer[Map[_,_]] { - def write(kryo: Kryo, output: Output, map: Map[_,_]) { - kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray) - } - - def read(kryo: Kryo, input: Input, tpe: Class[Map[_,_]]): Map[_,_] = { - kryo.readObject(input, classOf[Array[Any]]) - .sliding(2,2) - .map { case Array(k,v) => (k,v) } - .toMap - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 92a707ea57504..f43e98d614094 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -69,12 +69,36 @@ class DslQuerySuite extends QueryTest { checkAnswer( testData2.orderBy('a.desc, 'b.asc), Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + + checkAnswer( + arrayData.orderBy(GetItem('data, 0).asc), + arrayData.collect().sortBy(_.data(0)).toSeq) + + checkAnswer( + arrayData.orderBy(GetItem('data, 0).desc), + arrayData.collect().sortBy(_.data(0)).reverse.toSeq) + + checkAnswer( + mapData.orderBy(GetItem('data, 1).asc), + mapData.collect().sortBy(_.data(1)).toSeq) + + checkAnswer( + mapData.orderBy(GetItem('data, 1).desc), + mapData.collect().sortBy(_.data(1)).reverse.toSeq) } test("limit") { checkAnswer( testData.limit(10), testData.take(10).toSeq) + + checkAnswer( + arrayData.limit(1), + arrayData.take(1).toSeq) + + checkAnswer( + mapData.limit(1), + mapData.take(1).toSeq) } test("average") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 524549eb544fc..189dccd5253e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -85,6 +85,36 @@ class SQLQuerySuite extends QueryTest { checkAnswer( sql("SELECT * FROM testData2 ORDER BY a DESC, b ASC"), Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + + checkAnswer( + sql("SELECT * FROM arrayData ORDER BY data[0] ASC"), + arrayData.collect().sortBy(_.data(0)).toSeq) + + checkAnswer( + sql("SELECT * FROM arrayData ORDER BY data[0] DESC"), + arrayData.collect().sortBy(_.data(0)).reverse.toSeq) + + checkAnswer( + sql("SELECT * FROM mapData ORDER BY data[1] ASC"), + mapData.collect().sortBy(_.data(1)).toSeq) + + checkAnswer( + sql("SELECT * FROM mapData ORDER BY data[1] DESC"), + mapData.collect().sortBy(_.data(1)).reverse.toSeq) + } + + test("limit") { + checkAnswer( + sql("SELECT * FROM testData LIMIT 10"), + testData.take(10).toSeq) + + checkAnswer( + sql("SELECT * FROM arrayData LIMIT 1"), + arrayData.collect().take(1).toSeq) + + checkAnswer( + sql("SELECT * FROM mapData LIMIT 1"), + mapData.collect().take(1).toSeq) } test("average") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index aa71e274f7f4b..1aca3872524d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -74,6 +74,16 @@ object TestData { ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) arrayData.registerAsTable("arrayData") + case class MapData(data: Map[Int, String]) + val mapData = + TestSQLContext.sparkContext.parallelize( + MapData(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: + MapData(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: + MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: + MapData(Map(1 -> "a4", 2 -> "b4")) :: + MapData(Map(1 -> "a5")) :: Nil) + mapData.registerAsTable("mapData") + case class StringData(s: String) val repeatedData = TestSQLContext.sparkContext.parallelize(List.fill(2)(StringData("test"))) From 94c9d6f59859ebc77fae112c2c42c64b7a4d7f83 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 15 May 2014 11:21:33 -0700 Subject: [PATCH 74/82] [SPARK-1819] [SQL] Fix GetField.nullable. `GetField.nullable` should be `true` not only when `field.nullable` is `true` but also when `child.nullable` is `true`. Author: Takuya UESHIN Closes #757 from ueshin/issues/SPARK-1819 and squashes the following commits: 8781a11 [Takuya UESHIN] Modify a test to use named parameters. 5bfc77d [Takuya UESHIN] Fix GetField.nullable. --- .../sql/catalyst/expressions/complexTypes.scala | 2 +- .../expressions/ExpressionEvaluationSuite.scala | 13 +++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 195ca2eb3d589..b6aeae92f8bec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -74,7 +74,7 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio type EvaluatedType = Any def dataType = field.dataType - override def nullable = field.nullable + override def nullable = child.nullable || field.nullable override def foldable = child.foldable protected def structType = child.dataType match { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 344d8a304fc11..1132a30b42767 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -364,6 +364,19 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(GetField(BoundReference(2, AttributeReference("c", typeS)()), "a"), "aa", row) checkEvaluation(GetField(Literal(null, typeS), "a"), null, row) + + val typeS_notNullable = StructType( + StructField("a", StringType, nullable = false) + :: StructField("b", StringType, nullable = false) :: Nil + ) + + assert(GetField(BoundReference(2, + AttributeReference("c", typeS)()), "a").nullable === true) + assert(GetField(BoundReference(2, + AttributeReference("c", typeS_notNullable, nullable = false)()), "a").nullable === false) + + assert(GetField(Literal(null, typeS), "a").nullable === true) + assert(GetField(Literal(null, typeS_notNullable), "a").nullable === true) } test("arithmetic") { From d52761d67f42ad4d2ff02d96f0675fb3ab709f38 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 15 May 2014 11:59:59 -0700 Subject: [PATCH 75/82] [SPARK-1741][MLLIB] add predict(JavaRDD) to RegressionModel, ClassificationModel, and KMeans `model.predict` returns a RDD of Scala primitive type (Int/Double), which is recognized as Object in Java. Adding predict(JavaRDD) could make life easier for Java users. Added tests for KMeans, LinearRegression, and NaiveBayes. Will update examples after https://github.com/apache/spark/pull/653 gets merged. cc: @srowen Author: Xiangrui Meng Closes #670 from mengxr/predict-javardd and squashes the following commits: b77ccd8 [Xiangrui Meng] Merge branch 'master' into predict-javardd 43caac9 [Xiangrui Meng] add predict(JavaRDD) to RegressionModel, ClassificationModel, and KMeans --- .../classification/ClassificationModel.scala | 11 +++++++++- .../spark/mllib/clustering/KMeansModel.scala | 5 +++++ .../mllib/regression/RegressionModel.scala | 11 +++++++++- .../classification/JavaNaiveBayesSuite.java | 16 ++++++++++++++ .../mllib/clustering/JavaKMeansSuite.java | 14 +++++++++++++ .../regression/JavaLinearRegressionSuite.java | 21 +++++++++++++++++++ 6 files changed, 76 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 6332301e30cbd..b7a1d90d24d72 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -17,9 +17,10 @@ package org.apache.spark.mllib.classification +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD -import org.apache.spark.annotation.Experimental /** * :: Experimental :: @@ -43,4 +44,12 @@ trait ClassificationModel extends Serializable { * @return predicted category from the trained model */ def predict(testData: Vector): Double + + /** + * Predict values for examples stored in a JavaRDD. + * @param testData JavaRDD representing data points to be predicted + * @return a JavaRDD[java.lang.Double] where each entry contains the corresponding prediction + */ + def predict(testData: JavaRDD[Vector]): JavaRDD[java.lang.Double] = + predict(testData.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index ce14b06241932..fba21aefaaacd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.clustering +import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector @@ -40,6 +41,10 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser points.map(p => KMeans.findClosest(centersWithNorm, new BreezeVectorWithNorm(p))._1) } + /** Maps given points to their cluster indices. */ + def predict(points: JavaRDD[Vector]): JavaRDD[java.lang.Integer] = + predict(points.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Integer]] + /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this * model on the given data. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index b27e158b43f9a..64b02f7a6e7a9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -17,9 +17,10 @@ package org.apache.spark.mllib.regression +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.annotation.Experimental @Experimental trait RegressionModel extends Serializable { @@ -38,4 +39,12 @@ trait RegressionModel extends Serializable { * @return Double prediction from the trained model */ def predict(testData: Vector): Double + + /** + * Predict values for examples stored in a JavaRDD. + * @param testData JavaRDD representing data points to be predicted + * @return a JavaRDD[java.lang.Double] where each entry contains the corresponding prediction + */ + def predict(testData: JavaRDD[Vector]): JavaRDD[java.lang.Double] = + predict(testData.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index c80b1134ed1b2..743a43a139c0c 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -19,6 +19,8 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; import org.junit.After; @@ -87,4 +89,18 @@ public void runUsingStaticMethods() { int numAccurate2 = validatePrediction(POINTS, model2); Assert.assertEquals(POINTS.size(), numAccurate2); } + + @Test + public void testPredictJavaRDD() { + JavaRDD examples = sc.parallelize(POINTS, 2).cache(); + NaiveBayesModel model = NaiveBayes.train(examples.rdd()); + JavaRDD vectors = examples.map(new Function() { + @Override + public Vector call(LabeledPoint v) throws Exception { + return v.features(); + }}); + JavaRDD predictions = model.predict(vectors); + // Should be able to get the first prediction. + predictions.first(); + } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 49a614bd90cab..0c916ca378034 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -88,4 +88,18 @@ public void runKMeansUsingConstructor() { .run(data.rdd()); assertEquals(expectedCenter, model.clusterCenters()[0]); } + + @Test + public void testPredictJavaRDD() { + List points = Lists.newArrayList( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) + ); + JavaRDD data = sc.parallelize(points, 2); + KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); + JavaRDD predictions = model.predict(data); + // Should be able to get the first prediction. + predictions.first(); + } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 7151e553512b3..6dc6877691036 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -25,8 +25,10 @@ import org.junit.Before; import org.junit.Test; +import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.util.LinearDataGenerator; public class JavaLinearRegressionSuite implements Serializable { @@ -92,4 +94,23 @@ public void runLinearRegressionUsingStaticMethods() { Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } + @Test + public void testPredictJavaRDD() { + int nPoints = 100; + double A = 0.0; + double[] weights = {10, 10}; + JavaRDD testRDD = sc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); + LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); + JavaRDD vectors = testRDD.map(new Function() { + @Override + public Vector call(LabeledPoint v) throws Exception { + return v.features(); + } + }); + JavaRDD predictions = model.predict(vectors); + // Should be able to get the first prediction. + predictions.first(); + } } From 08e7606a964e3d1ac1d565f33651ff0035c75044 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 15 May 2014 16:35:39 -0700 Subject: [PATCH 76/82] SPARK-1851. Upgrade Avro dependency to 1.7.6 so Spark can read Avro file... ...s Author: Sandy Ryza Closes #795 from sryza/sandy-spark-1851 and squashes the following commits: 79c8227 [Sandy Ryza] SPARK-1851. Upgrade Avro dependency to 1.7.6 so Spark can read Avro files --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 786b6d4984d86..5df7036585b75 100644 --- a/pom.xml +++ b/pom.xml @@ -129,7 +129,7 @@ 8.1.14.v20131031 0.3.6 3.0.0 - 1.7.4 + 1.7.6 0.7.1 64m From e66e31be51f396c8f6b7a45119b8b31c4d8cdf79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stevo=20Slavi=C4=87?= Date: Thu, 15 May 2014 16:44:14 -0700 Subject: [PATCH 77/82] SPARK-1803 Replaced colon in filenames with a dash MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This patch replaces colon in several filenames with dash to make these filenames Windows compatible. Author: Stevo Slavić Author: Stevo Slavic Closes #739 from sslavic/SPARK-1803 and squashes the following commits: 3ec66eb [Stevo Slavic] Removed extra empty line which was causing test to fail b967cc3 [Stevo Slavić] Aligned tests and names of test resources 2b12776 [Stevo Slavić] Fixed a typo in file name 1c5dfff [Stevo Slavić] Replaced colon in file name with dash 8f5bf7f [Stevo Slavić] Replaced colon in file name with dash c5b5083 [Stevo Slavić] Replaced colon in file name with dash a49801f [Stevo Slavić] Replaced colon in file name with dash 401d99e [Stevo Slavić] Replaced colon in file name with dash 40a9621 [Stevo Slavić] Replaced colon in file name with dash 4774580 [Stevo Slavić] Replaced colon in file name with dash 004f8bb [Stevo Slavić] Replaced colon in file name with dash d6a3e2c [Stevo Slavić] Replaced colon in file name with dash b585126 [Stevo Slavić] Replaced colon in file name with dash 028e48a [Stevo Slavić] Replaced colon in file name with dash ece0507 [Stevo Slavić] Replaced colon in file name with dash 84f5d2f [Stevo Slavić] Replaced colon in file name with dash 2fc7854 [Stevo Slavić] Replaced colon in file name with dash 9e1467d [Stevo Slavić] Replaced colon in file name with dash --- ...y test-0-cb0737d268260da971487a8ac5e3d8be} | 0 ...y test-0-c89dcddb8539bae211ea28cd1e695e35} | 0 ...y test-0-9589ac54d6de575083765ee64051ebaa} | 0 ...y test-0-d6d2726928df920ab99078015b8ff494} | 0 ...ry test-0-cfe78c68302bd143d989c383c3b0477} | 0 ...y test-0-358525951e893ed9686c46bf1f7447ee} | 0 ...y test-0-d9d50146537344742300ce7b52c18c91} | 0 ...y test-0-c25cdafe7d85c42a9d9cd559d88a6049} | 0 ...y test-0-63ab78355dbc6fa0d7619c8a35b0f7a7} | 0 ...y test-0-3adc3a7f76b2abd059904ba81a595db3} | 0 ...y test-0-681332efaff7b12e3ca12d9f021cd344} | 0 ...y test-0-3893ec7a86062215d9f817f1495a69bb} | 0 ...y test-0-75148567b91227053f728d72f7dc7c10} | 0 ...y test-0-4e4d2651e1e6e3c224242078d0201190} | 0 ...y test-0-b6e59464383591f02408f8765ac6a5d5} | 0 .../sql/hive/execution/PruningSuite.scala | 30 +++++++++---------- 16 files changed, 15 insertions(+), 15 deletions(-) rename sql/hive/src/test/resources/golden/{Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be => Column pruning - filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 => Column pruning - non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa => Column pruning - projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 => Column pruning - simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 => Column pruning - with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee => Column pruning - with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 => Column pruning - with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91} (100%) rename sql/hive/src/test/resources/golden/{Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 => Column pruning - without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 => Partition pruning - all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 => Partition pruning - left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3} (100%) rename sql/hive/src/test/resources/golden/{Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 => Partition pruning - non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb => Partition pruning - non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 => Partition pruning - pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 => Partition pruning - with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190} (100%) rename sql/hive/src/test/resources/golden/{Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 => Partition pruning - with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5} (100%) diff --git a/sql/hive/src/test/resources/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be b/sql/hive/src/test/resources/golden/Column pruning - filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be rename to sql/hive/src/test/resources/golden/Column pruning - filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be diff --git a/sql/hive/src/test/resources/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 b/sql/hive/src/test/resources/golden/Column pruning - non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 rename to sql/hive/src/test/resources/golden/Column pruning - non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 diff --git a/sql/hive/src/test/resources/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa b/sql/hive/src/test/resources/golden/Column pruning - projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa rename to sql/hive/src/test/resources/golden/Column pruning - projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa diff --git a/sql/hive/src/test/resources/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 b/sql/hive/src/test/resources/golden/Column pruning - simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 rename to sql/hive/src/test/resources/golden/Column pruning - simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 diff --git a/sql/hive/src/test/resources/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 b/sql/hive/src/test/resources/golden/Column pruning - with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 rename to sql/hive/src/test/resources/golden/Column pruning - with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 diff --git a/sql/hive/src/test/resources/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee b/sql/hive/src/test/resources/golden/Column pruning - with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee rename to sql/hive/src/test/resources/golden/Column pruning - with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee diff --git a/sql/hive/src/test/resources/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 b/sql/hive/src/test/resources/golden/Column pruning - with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 rename to sql/hive/src/test/resources/golden/Column pruning - with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 diff --git a/sql/hive/src/test/resources/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 b/sql/hive/src/test/resources/golden/Column pruning - without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 similarity index 100% rename from sql/hive/src/test/resources/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 rename to sql/hive/src/test/resources/golden/Column pruning - without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 b/sql/hive/src/test/resources/golden/Partition pruning - all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 rename to sql/hive/src/test/resources/golden/Partition pruning - all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 b/sql/hive/src/test/resources/golden/Partition pruning - left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 rename to sql/hive/src/test/resources/golden/Partition pruning - left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 diff --git a/sql/hive/src/test/resources/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 b/sql/hive/src/test/resources/golden/Partition pruning - non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 similarity index 100% rename from sql/hive/src/test/resources/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 rename to sql/hive/src/test/resources/golden/Partition pruning - non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb b/sql/hive/src/test/resources/golden/Partition pruning - non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb rename to sql/hive/src/test/resources/golden/Partition pruning - non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb diff --git a/sql/hive/src/test/resources/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 b/sql/hive/src/test/resources/golden/Partition pruning - pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 rename to sql/hive/src/test/resources/golden/Partition pruning - pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 b/sql/hive/src/test/resources/golden/Partition pruning - with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 rename to sql/hive/src/test/resources/golden/Partition pruning - with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 diff --git a/sql/hive/src/test/resources/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 b/sql/hive/src/test/resources/golden/Partition pruning - with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 similarity index 100% rename from sql/hive/src/test/resources/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 rename to sql/hive/src/test/resources/golden/Partition pruning - with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 25eca397462f7..34434449a0d77 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -28,7 +28,7 @@ import scala.collection.JavaConversions._ class PruningSuite extends HiveComparisonTest { // Column pruning tests - createPruningTest("Column pruning: with partitioned table", + createPruningTest("Column pruning - with partitioned table", "SELECT key FROM srcpart WHERE ds = '2008-04-08' LIMIT 3", Seq("key"), Seq("key"), @@ -36,43 +36,43 @@ class PruningSuite extends HiveComparisonTest { Seq("2008-04-08", "11"), Seq("2008-04-08", "12"))) - createPruningTest("Column pruning: with non-partitioned table", + createPruningTest("Column pruning - with non-partitioned table", "SELECT key FROM src WHERE key > 10 LIMIT 3", Seq("key"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: with multiple projects", + createPruningTest("Column pruning - with multiple projects", "SELECT c1 FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3", Seq("c1"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: projects alias substituting", + createPruningTest("Column pruning - projects alias substituting", "SELECT c1 AS c2 FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3", Seq("c2"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: filter alias in-lining", + createPruningTest("Column pruning - filter alias in-lining", "SELECT c1 FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 WHERE c1 < 100 LIMIT 3", Seq("c1"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: without filters", + createPruningTest("Column pruning - without filters", "SELECT c1 FROM (SELECT key AS c1 FROM src) t1 LIMIT 3", Seq("c1"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: simple top project without aliases", + createPruningTest("Column pruning - simple top project without aliases", "SELECT key FROM (SELECT key FROM src WHERE key > 10) t1 WHERE key < 100 LIMIT 3", Seq("key"), Seq("key"), Seq.empty) - createPruningTest("Column pruning: non-trivial top project with aliases", + createPruningTest("Column pruning - non-trivial top project with aliases", "SELECT c1 * 2 AS double FROM (SELECT key AS c1 FROM src WHERE key > 10) t1 LIMIT 3", Seq("double"), Seq("key"), @@ -80,19 +80,19 @@ class PruningSuite extends HiveComparisonTest { // Partition pruning tests - createPruningTest("Partition pruning: non-partitioned, non-trivial project", + createPruningTest("Partition pruning - non-partitioned, non-trivial project", "SELECT key * 2 AS double FROM src WHERE value IS NOT NULL", Seq("double"), Seq("key", "value"), Seq.empty) - createPruningTest("Partiton pruning: non-partitioned table", + createPruningTest("Partition pruning - non-partitioned table", "SELECT value FROM src WHERE key IS NOT NULL", Seq("value"), Seq("value", "key"), Seq.empty) - createPruningTest("Partition pruning: with filter on string partition key", + createPruningTest("Partition pruning - with filter on string partition key", "SELECT value, hr FROM srcpart1 WHERE ds = '2008-04-08'", Seq("value", "hr"), Seq("value", "hr"), @@ -100,7 +100,7 @@ class PruningSuite extends HiveComparisonTest { Seq("2008-04-08", "11"), Seq("2008-04-08", "12"))) - createPruningTest("Partition pruning: with filter on int partition key", + createPruningTest("Partition pruning - with filter on int partition key", "SELECT value, hr FROM srcpart1 WHERE hr < 12", Seq("value", "hr"), Seq("value", "hr"), @@ -108,20 +108,20 @@ class PruningSuite extends HiveComparisonTest { Seq("2008-04-08", "11"), Seq("2008-04-09", "11"))) - createPruningTest("Partition pruning: left only 1 partition", + createPruningTest("Partition pruning - left only 1 partition", "SELECT value, hr FROM srcpart1 WHERE ds = '2008-04-08' AND hr < 12", Seq("value", "hr"), Seq("value", "hr"), Seq( Seq("2008-04-08", "11"))) - createPruningTest("Partition pruning: all partitions pruned", + createPruningTest("Partition pruning - all partitions pruned", "SELECT value, hr FROM srcpart1 WHERE ds = '2014-01-27' AND hr = 11", Seq("value", "hr"), Seq("value", "hr"), Seq.empty) - createPruningTest("Partition pruning: pruning with both column key and partition key", + createPruningTest("Partition pruning - pruning with both column key and partition key", "SELECT value, hr FROM srcpart1 WHERE value IS NOT NULL AND hr < 12", Seq("value", "hr"), Seq("value", "hr"), From a4aafe5f9fb191533400caeafddf04986492c95f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 15 May 2014 16:50:42 -0700 Subject: [PATCH 78/82] [SQL] Fix tiny/small ints from HiveMetastore. Author: Michael Armbrust Closes #797 from marmbrus/smallInt and squashes the following commits: 2db9dae [Michael Armbrust] Fix tiny/small ints from HiveMetastore. --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index fa30f596f8c67..636c4f7b93190 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -183,7 +183,8 @@ object HiveMetastoreTypes extends RegexParsers { "string" ^^^ StringType | "float" ^^^ FloatType | "int" ^^^ IntegerType | - "tinyint" ^^^ ShortType | + "tinyint" ^^^ ByteType | + "smallint" ^^^ ShortType | "double" ^^^ DoubleType | "bigint" ^^^ LongType | "binary" ^^^ BinaryType | @@ -227,7 +228,8 @@ object HiveMetastoreTypes extends RegexParsers { case StringType => "string" case FloatType => "float" case IntegerType => "int" - case ShortType =>"tinyint" + case ByteType => "tinyint" + case ShortType => "smallint" case DoubleType => "double" case LongType => "bigint" case BinaryType => "binary" From e1e3416c4e5f6f32983597d74866dbb809cf6a5e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 15 May 2014 16:58:37 -0700 Subject: [PATCH 79/82] Fixes a misplaced comment. Fixes a misplaced comment from #785. @pwendell Author: Prashant Sharma Closes #788 from ScrapCodes/patch-1 and squashes the following commits: 3ef6a69 [Prashant Sharma] Update package-info.java 67d9461 [Prashant Sharma] Update package-info.java --- .../main/scala/org/apache/spark/graphx/lib/package-info.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java b/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java index 49ad7263141ce..e987ee8560ff7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/package-info.java @@ -16,6 +16,6 @@ */ /** - * Internal support for MLLib Python API. + * Various analytics functions for graphs. */ -package org.apache.spark.graphx.lib; \ No newline at end of file +package org.apache.spark.graphx.lib; From 94c5139607ec876782e594012a108ebf55fa97db Mon Sep 17 00:00:00 2001 From: Huajian Mao Date: Thu, 15 May 2014 18:20:16 -0700 Subject: [PATCH 80/82] Typos in Spark Author: Huajian Mao Closes #798 from huajianmao/patch-1 and squashes the following commits: 208a454 [Huajian Mao] A typo in Task 1b515af [Huajian Mao] A typo in the message --- core/src/main/scala/org/apache/spark/scheduler/Task.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 2ca3479c80efc..5871edeb856ad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.ByteBufferInputStream * - [[org.apache.spark.scheduler.ResultTask]] * * A Spark job consists of one or more stages. The very last stage in a job consists of multiple - * ResultTask's, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task + * ResultTasks, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task * and sends the task output back to the driver application. A ShuffleMapTask executes the task * and divides the task output to multiple buckets (based on the task's partitioner). * diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 649eed213e09a..17292b4c15b8b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -105,7 +105,7 @@ private[spark] class TaskSchedulerImpl( SchedulingMode.withName(schedulingModeConf.toUpperCase) } catch { case e: java.util.NoSuchElementException => - throw new SparkException(s"Urecognized spark.scheduler.mode: $schedulingModeConf") + throw new SparkException(s"Unrecognized spark.scheduler.mode: $schedulingModeConf") } // This is a var so that we can reset it for testing purposes. From bb98ecafce196ecc5bc3a1e4cc9264df7b752c6a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 15 May 2014 21:37:58 -0700 Subject: [PATCH 81/82] SPARK-1860: Do not cleanup application work/ directories by default This causes an unrecoverable error for applications that are running for longer than 7 days that have jars added to the SparkContext, as the jars are cleaned up even though the application is still running. Author: Aaron Davidson Closes #800 from aarondav/shitty-defaults and squashes the following commits: a573fbb [Aaron Davidson] SPARK-1860: Do not cleanup application work/ directories by default --- .../main/scala/org/apache/spark/deploy/worker/Worker.scala | 2 +- docs/configuration.md | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 134624c35a57e..fb9cc116cd08b 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -65,7 +65,7 @@ private[spark] class Worker( val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 - val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", true) + val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up diff --git a/docs/configuration.md b/docs/configuration.md index 2eed96f704a4f..f89040d714773 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -390,10 +390,11 @@ Apart from these, the following properties are also available, and may be useful spark.worker.cleanup.enabled - true + false Enable periodic cleanup of worker / application directories. Note that this only affects standalone - mode, as YARN works differently. + mode, as YARN works differently. Applications directories are cleaned up regardless of whether + the application is still running. From a20fea98811d98958567780815fcf0d4fb4e28d4 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 15 May 2014 22:12:34 -0700 Subject: [PATCH 82/82] [Spark-1461] Deferred Expression Evaluation (short-circuit evaluation) This patch unify the foldable & nullable interface for Expression. 1) Deterministic-less UDF (like Rand()) can not be folded. 2) Short-circut will significantly improves the performance in Expression Evaluation, however, the stateful UDF should not be ignored in a short-circuit evaluation(e.g. in expression: col1 > 0 and row_sequence() < 1000, row_sequence() can not be ignored even if col1 > 0 is false) I brought an concept of DeferredObject from Hive, which has 2 kinds of children classes (EagerResult / DeferredResult), the former requires triggering the evaluation before it's created, while the later trigger the evaluation when first called its get() method. Author: Cheng Hao Closes #446 from chenghao-intel/expression_deferred_evaluation and squashes the following commits: d2729de [Cheng Hao] Fix the codestyle issues a08f09c [Cheng Hao] fix bug in or/and short-circuit evaluation af2236b [Cheng Hao] revert the short-circuit expression evaluation for IF b7861d2 [Cheng Hao] Add Support for Deferred Expression Evaluation --- .../sql/catalyst/expressions/predicates.scala | 47 +++++++++++++------ .../org/apache/spark/sql/hive/hiveUdfs.scala | 28 ++++++++--- 2 files changed, 53 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 6ee479939d25c..d111578530506 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -98,13 +98,19 @@ case class And(left: Expression, right: Expression) extends BinaryPredicate { override def eval(input: Row): Any = { val l = left.eval(input) - val r = right.eval(input) - if (l == false || r == false) { - false - } else if (l == null || r == null ) { - null + if (l == false) { + false } else { - true + val r = right.eval(input) + if (r == false) { + false + } else { + if (l != null && r != null) { + true + } else { + null + } + } } } } @@ -114,13 +120,19 @@ case class Or(left: Expression, right: Expression) extends BinaryPredicate { override def eval(input: Row): Any = { val l = left.eval(input) - val r = right.eval(input) - if (l == true || r == true) { + if (l == true) { true - } else if (l == null || r == null) { - null } else { - false + val r = right.eval(input) + if (r == true) { + true + } else { + if (l != null && r != null) { + false + } else { + null + } + } } } } @@ -133,8 +145,12 @@ case class Equals(left: Expression, right: Expression) extends BinaryComparison def symbol = "=" override def eval(input: Row): Any = { val l = left.eval(input) - val r = right.eval(input) - if (l == null || r == null) null else l == r + if (l == null) { + null + } else { + val r = right.eval(input) + if (r == null) null else l == r + } } } @@ -162,7 +178,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi extends Expression { def children = predicate :: trueValue :: falseValue :: Nil - def nullable = trueValue.nullable || falseValue.nullable + override def nullable = trueValue.nullable || falseValue.nullable def references = children.flatMap(_.references).toSet override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType def dataType = { @@ -175,8 +191,9 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } type EvaluatedType = Any + override def eval(input: Row): Any = { - if (predicate.eval(input).asInstanceOf[Boolean]) { + if (true == predicate.eval(input)) { trueValue.eval(input) } else { falseValue.eval(input) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index d50e2c65b7b36..572902042337f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -248,17 +248,31 @@ private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) isUDFDeterministic && children.foldLeft(true)((prev, n) => prev && n.foldable) } + protected lazy val deferedObjects = Array.fill[DeferredObject](children.length)({ + new DeferredObjectAdapter + }) + + // Adapter from Catalyst ExpressionResult to Hive DeferredObject + class DeferredObjectAdapter extends DeferredObject { + private var func: () => Any = _ + def set(func: () => Any) { + this.func = func + } + override def prepare(i: Int) = {} + override def get(): AnyRef = wrap(func()) + } + val dataType: DataType = inspectorToDataType(returnInspector) override def eval(input: Row): Any = { returnInspector // Make sure initialized. - val args = children.map { v => - new DeferredObject { - override def prepare(i: Int) = {} - override def get(): AnyRef = wrap(v.eval(input)) - } - }.toArray - unwrap(function.evaluate(args)) + var i = 0 + while (i < children.length) { + val idx = i + deferedObjects(i).asInstanceOf[DeferredObjectAdapter].set(() => {children(idx).eval(input)}) + i += 1 + } + unwrap(function.evaluate(deferedObjects)) } }