From 16788a654246067fd966033b5dc9bc0d4c759b70 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 11 Mar 2014 11:16:59 -0700 Subject: [PATCH 01/23] SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues... This patch removes Ganglia integration from the default build. It allows users willing to link against LGPL code to use Ganglia by adding build flags or linking against a new Spark artifact called spark-ganglia-lgpl. This brings Spark in line with the Apache policy on LGPL code enumerated here: https://www.apache.org/legal/3party.html#options-optional Author: Patrick Wendell Closes #108 from pwendell/ganglia and squashes the following commits: 326712a [Patrick Wendell] Responding to review feedback 5f28ee4 [Patrick Wendell] SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues. --- assembly/pom.xml | 10 +++++ core/pom.xml | 4 -- dev/audit-release/README.md | 11 +++++ .../src/main/scala/SparkApp.scala | 15 ++++++- dev/audit-release/sbt_app_ganglia/build.sbt | 31 +++++++++++++ .../src/main/scala/SparkApp.scala | 39 ++++++++++++++++ dev/create-release/create-release.sh | 4 +- docs/monitoring.md | 13 +++++- extras/spark-ganglia-lgpl/pom.xml | 45 +++++++++++++++++++ .../spark/metrics/sink/GangliaSink.scala | 0 pom.xml | 9 +++- project/SparkBuild.scala | 25 ++++++++--- 12 files changed, 190 insertions(+), 16 deletions(-) create mode 100644 dev/audit-release/README.md create mode 100644 dev/audit-release/sbt_app_ganglia/build.sbt create mode 100644 dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala create mode 100644 extras/spark-ganglia-lgpl/pom.xml rename {core => extras/spark-ganglia-lgpl}/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala (100%) diff --git a/assembly/pom.xml b/assembly/pom.xml index 82a5985504b4e..22bbbc57d81d4 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -158,6 +158,16 @@ + + spark-ganglia-lgpl + + + org.apache.spark + spark-ganglia-lgpl_${scala.binary.version} + ${project.version} + + + bigtop-dist + + 4.0.0 + + org.apache.spark + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + + org.apache.spark + spark-ganglia-lgpl_2.10 + jar + Spark Ganglia Integration + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + + com.codahale.metrics + metrics-ganglia + + + diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala rename to extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala diff --git a/pom.xml b/pom.xml index f0c877dcfe7b2..986626f029d75 100644 --- a/pom.xml +++ b/pom.xml @@ -756,12 +756,19 @@ 0.23.7 - yarn + + + + spark-ganglia-lgpl + + extras/spark-ganglia-lgpl + + java8-tests diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8fa220c413291..b0c3bf29dfd4f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -65,7 +65,7 @@ object SparkBuild extends Build { lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) - .dependsOn(core, graphx, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) + .dependsOn(core, graphx, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*) lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects") @@ -91,19 +91,26 @@ object SparkBuild extends Build { lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() - // Conditionally include the java 8 sub-project + // Include Ganglia integration if the user has enabled Ganglia + // This is isolated from the normal build due to LGPL-licensed code in the library + lazy val isGangliaEnabled = Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined + lazy val gangliaProj = Project("spark-ganglia-lgpl", file("extras/spark-ganglia-lgpl"), settings = gangliaSettings).dependsOn(core) + val maybeGanglia: Seq[ClasspathDependency] = if (isGangliaEnabled) Seq(gangliaProj) else Seq() + val maybeGangliaRef: Seq[ProjectReference] = if (isGangliaEnabled) Seq(gangliaProj) else Seq() + + // Include the Java 8 project if the JVM version is 8+ lazy val javaVersion = System.getProperty("java.specification.version") lazy val isJava8Enabled = javaVersion.toDouble >= "1.8".toDouble val maybeJava8Tests = if (isJava8Enabled) Seq[ProjectReference](java8Tests) else Seq[ProjectReference]() lazy val java8Tests = Project("java8-tests", file("extras/java8-tests"), settings = java8TestsSettings). dependsOn(core) dependsOn(streaming % "compile->compile;test->test") - // Conditionally include the yarn sub-project + // Include the YARN project if the user has enabled YARN lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core) lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core) - lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn else yarnAlpha) else Seq[ClasspathDependency]() - lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn else yarnAlpha) else Seq[ProjectReference]() + lazy val maybeYarn: Seq[ClasspathDependency] = if (isYarnEnabled) Seq(if (isNewHadoop) yarn else yarnAlpha) else Seq() + lazy val maybeYarnRef: Seq[ProjectReference] = if (isYarnEnabled) Seq(if (isNewHadoop) yarn else yarnAlpha) else Seq() lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings) .dependsOn(streaming % "compile->compile;test->test") @@ -127,7 +134,7 @@ object SparkBuild extends Build { .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) // Everything except assembly, tools, java8Tests and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ maybeYarnRef + lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ maybeYarnRef ++ maybeGangliaRef lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests @@ -296,7 +303,6 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", - "com.codahale.metrics" % "metrics-ganglia" % "3.0.0", "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), @@ -384,6 +390,11 @@ object SparkBuild extends Build { name := "spark-yarn" ) + def gangliaSettings = sharedSettings ++ Seq( + name := "spark-ganglia-lgpl", + libraryDependencies += "com.codahale.metrics" % "metrics-ganglia" % "3.0.0" + ) + def java8TestsSettings = sharedSettings ++ Seq( name := "java8-tests", javacOptions := Seq("-target", "1.8", "-source", "1.8"), From 2409af9dcf238e1ad87080a389e05a696c41dc72 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 11 Mar 2014 22:39:17 -0700 Subject: [PATCH 02/23] SPARK-1064 This reopens PR 649 from incubator-spark against the new repo Author: Sandy Ryza Closes #102 from sryza/sandy-spark-1064 and squashes the following commits: 270e490 [Sandy Ryza] Handle different application classpath variables in different versions 88b04e0 [Sandy Ryza] SPARK-1064. Make it possible to run on YARN without bundling Hadoop jars in Spark assembly --- docs/building-with-maven.md | 6 +++ pom.xml | 46 +++++++++++++++++++ .../apache/spark/deploy/yarn/ClientBase.scala | 43 ++++++++++++++++- 3 files changed, 94 insertions(+), 1 deletion(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index d3bc34e68b240..730a6e7932564 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -88,3 +88,9 @@ Running only java 8 tests and nothing else. Java 8 tests are run when -Pjava8-tests profile is enabled, they will run in spite of -DskipTests. 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. + +## 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/pom.xml b/pom.xml index 986626f029d75..d2c36dc260fa4 100644 --- a/pom.xml +++ b/pom.xml @@ -807,5 +807,51 @@ + + + + hadoop-provided + + false + + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + org.apache.hadoop + hadoop-yarn-client + provided + + + org.apache.avro + avro + provided + + + org.apache.avro + avro-ipc + provided + + + org.apache.zookeeper + zookeeper + provided + + + + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 4b6c7db836b02..74de4293d9096 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -29,8 +29,10 @@ import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.mapred.Master +import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords._ @@ -379,9 +381,48 @@ object ClientBase { // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { - for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) { + val classpathEntries = Option(conf.getStrings( + YarnConfiguration.YARN_APPLICATION_CLASSPATH)).getOrElse( + getDefaultYarnApplicationClasspath()) + for (c <- classpathEntries) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim) } + + val mrClasspathEntries = Option(conf.getStrings( + "mapreduce.application.classpath")).getOrElse( + getDefaultMRApplicationClasspath()) + if (mrClasspathEntries != null) { + for (c <- mrClasspathEntries) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim) + } + } + } + + def getDefaultYarnApplicationClasspath(): Array[String] = { + try { + val field = classOf[MRJobConfig].getField("DEFAULT_YARN_APPLICATION_CLASSPATH") + field.get(null).asInstanceOf[Array[String]] + } catch { + case err: NoSuchFieldError => null + } + } + + /** + * In Hadoop 0.23, the MR application classpath comes with the YARN application + * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. + * So we need to use reflection to retrieve it. + */ + def getDefaultMRApplicationClasspath(): Array[String] = { + try { + val field = classOf[MRJobConfig].getField("DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH") + if (field.getType == classOf[String]) { + StringUtils.getStrings(field.get(null).asInstanceOf[String]) + } else { + field.get(null).asInstanceOf[Array[String]] + } + } catch { + case err: NoSuchFieldError => null + } } def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) { From af7f2f10902c7b42e08797f7467dd06e4803594c Mon Sep 17 00:00:00 2001 From: prabinb Date: Tue, 11 Mar 2014 23:57:05 -0700 Subject: [PATCH 03/23] Spark-1163, Added missing Python RDD functions Author: prabinb Closes #92 from prabinb/python-api-rdd and squashes the following commits: 51129ca [prabinb] Added missing Python RDD functions Added __repr__ function to StorageLevel class. Added doctest for RDD.getStorageLevel(). --- python/pyspark/rdd.py | 42 ++++++++++++++++++++++++++++++++++ python/pyspark/storagelevel.py | 4 ++++ 2 files changed, 46 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 39916d21c76c5..0f28dbd6fcc09 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -36,6 +36,7 @@ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler +from pyspark.storagelevel import StorageLevel from py4j.java_collections import ListConverter, MapConverter @@ -1119,6 +1120,47 @@ def zip(self, other): other._jrdd_deserializer) return RDD(pairRDD, self.ctx, deserializer) + def name(self): + """ + Return the name of this RDD. + """ + name_ = self._jrdd.name() + if not name_: + return None + return name_.encode('utf-8') + + def setName(self, name): + """ + Assign a name to this RDD. + >>> rdd1 = sc.parallelize([1,2]) + >>> rdd1.setName('RDD1') + >>> rdd1.name() + 'RDD1' + """ + self._jrdd.setName(name) + + def toDebugString(self): + """ + A description of this RDD and its recursive dependencies for debugging. + """ + debug_string = self._jrdd.toDebugString() + if not debug_string: + return None + return debug_string.encode('utf-8') + + def getStorageLevel(self): + """ + Get the RDD's current storage level. + >>> rdd1 = sc.parallelize([1,2]) + >>> rdd1.getStorageLevel() + StorageLevel(False, False, False, 1) + """ + java_storage_level = self._jrdd.getStorageLevel() + storage_level = StorageLevel(java_storage_level.useDisk(), + java_storage_level.useMemory(), + java_storage_level.deserialized(), + java_storage_level.replication()) + return storage_level # TODO: `lookup` is disabled because we can't make direct comparisons based # on the key; we need to compare the hash of the key to the hash of the diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index b31f4762e69bc..c3e3a44e8e7ab 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -31,6 +31,10 @@ def __init__(self, useDisk, useMemory, deserialized, replication = 1): self.deserialized = deserialized self.replication = replication + def __repr__(self): + return "StorageLevel(%s, %s, %s, %s)" % ( + self.useDisk, self.useMemory, self.deserialized, self.replication) + StorageLevel.DISK_ONLY = StorageLevel(True, False, False) StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, 2) StorageLevel.MEMORY_ONLY = StorageLevel(False, True, True) From c8c59b326e587b7515ba4f43399c67d949df244f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 12 Mar 2014 10:32:01 -0700 Subject: [PATCH 04/23] [SPARK-1232] Fix the hadoop 0.23 yarn build Author: Thomas Graves Closes #127 from tgravescs/SPARK-1232 and squashes the following commits: c05cfd4 [Thomas Graves] Fix the hadoop 0.23 yarn build --- core/pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 2248f9d0446c0..a6f478b09bda0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -30,6 +30,18 @@ jar Spark Project Core http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + From b5162f4426268fdd3acc592e636b8a59c478db9f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 12 Mar 2014 11:25:41 -0700 Subject: [PATCH 05/23] [SPARK-1233] Fix running hadoop 0.23 due to java.lang.NoSuchFieldException: DEFAULT_M... ...APREDUCE_APPLICATION_CLASSPATH Author: Thomas Graves Closes #129 from tgravescs/SPARK-1233 and squashes the following commits: 85ff5a6 [Thomas Graves] Fix running hadoop 0.23 due to java.lang.NoSuchFieldException: DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 74de4293d9096..74c5e0f18e7bd 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -404,6 +404,7 @@ object ClientBase { field.get(null).asInstanceOf[Array[String]] } catch { case err: NoSuchFieldError => null + case err: NoSuchFieldException => null } } @@ -422,6 +423,7 @@ object ClientBase { } } catch { case err: NoSuchFieldError => null + case err: NoSuchFieldException => null } } From 5d1ec64e7934ad7f922cdab516fa5de690644780 Mon Sep 17 00:00:00 2001 From: liguoqiang Date: Wed, 12 Mar 2014 12:59:51 -0700 Subject: [PATCH 06/23] Fix #SPARK-1149 Bad partitioners can cause Spark to hang Author: liguoqiang Closes #44 from witgo/SPARK-1149 and squashes the following commits: 3dcdcaf [liguoqiang] Merge branch 'master' into SPARK-1149 8425395 [liguoqiang] Merge remote-tracking branch 'upstream/master' into SPARK-1149 3dad595 [liguoqiang] review comment e3e56aa [liguoqiang] Merge branch 'master' into SPARK-1149 b0d5c07 [liguoqiang] review comment d0a6005 [liguoqiang] review comment 3395ee7 [liguoqiang] Merge remote-tracking branch 'upstream/master' into SPARK-1149 ac006a3 [liguoqiang] code Formatting 3feb3a8 [liguoqiang] Merge branch 'master' into SPARK-1149 adc443e [liguoqiang] partitions check bugfix 928e1e3 [liguoqiang] Added a unit test for PairRDDFunctions.lookup with bad partitioner db6ecc5 [liguoqiang] Merge branch 'master' into SPARK-1149 1e3331e [liguoqiang] Merge branch 'master' into SPARK-1149 3348619 [liguoqiang] Optimize performance for partitions check 61e5a87 [liguoqiang] Merge branch 'master' into SPARK-1149 e68210a [liguoqiang] add partition index check to submitJob 3a65903 [liguoqiang] make the code more readable 6bb725e [liguoqiang] fix #SPARK-1149 Bad partitioners can cause Spark to hang --- .../scala/org/apache/spark/SparkContext.scala | 6 ++++++ .../apache/spark/rdd/PairRDDFunctionsSuite.scala | 16 ++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 745e3fa4e85f6..852ed8fe1fb91 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -852,6 +852,9 @@ class SparkContext( partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { + partitions.foreach{ p => + require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") + } val callSite = getCallSite val cleanedFunc = clean(func) logInfo("Starting job: " + callSite) @@ -955,6 +958,9 @@ class SparkContext( resultHandler: (Int, U) => Unit, resultFunc: => R): SimpleFutureAction[R] = { + partitions.foreach{ p => + require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") + } val cleanF = clean(processPartition) val callSite = getCallSite val waiter = dagScheduler.submitJob( 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 85e8eb5dc3a1e..f9e994b13dfbc 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -373,6 +373,22 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { assert(shuffled.lookup(5) === Seq(6,7)) assert(shuffled.lookup(-1) === Seq()) } + + test("lookup with bad partitioner") { + val pairs = sc.parallelize(Array((1,2), (3,4), (5,6), (5,7))) + + val p = new Partitioner { + def numPartitions: Int = 2 + + def getPartition(key: Any): Int = key.hashCode() % 2 + } + val shuffled = pairs.partitionBy(p) + + assert(shuffled.partitioner === Some(p)) + assert(shuffled.lookup(1) === Seq(2)) + intercept[IllegalArgumentException] {shuffled.lookup(-1)} + } + } /* From b8afe3052086547879ebf28d6e36207e0d370710 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 12 Mar 2014 15:57:44 -0700 Subject: [PATCH 07/23] SPARK-1162 Added top in python. Author: Prashant Sharma Closes #93 from ScrapCodes/SPARK-1162/pyspark-top-takeOrdered and squashes the following commits: ece1fa4 [Prashant Sharma] Added top in python. --- python/pyspark/rdd.py | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 0f28dbd6fcc09..6d549b40e5698 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -29,6 +29,7 @@ from tempfile import NamedTemporaryFile from threading import Thread import warnings +from heapq import heappush, heappop, heappushpop from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long @@ -660,6 +661,30 @@ def mergeMaps(m1, m2): m1[k] += v return m1 return self.mapPartitions(countPartition).reduce(mergeMaps) + + def top(self, num): + """ + Get the top N elements from a RDD. + + Note: It returns the list sorted in ascending order. + >>> sc.parallelize([10, 4, 2, 12, 3]).top(1) + [12] + >>> sc.parallelize([2, 3, 4, 5, 6]).cache().top(2) + [5, 6] + """ + def topIterator(iterator): + q = [] + for k in iterator: + if len(q) < num: + heappush(q, k) + else: + heappushpop(q, k) + yield q + + def merge(a, b): + return next(topIterator(a + b)) + + return sorted(self.mapPartitions(topIterator).reduce(merge)) def take(self, num): """ From 9032f7c0d5f1ae7985a20d54ca04c297201aae85 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 12 Mar 2014 17:43:12 -0700 Subject: [PATCH 08/23] SPARK-1160: Deprecate toArray in RDD https://spark-project.atlassian.net/browse/SPARK-1160 reported by @mateiz: "It's redundant with collect() and the name doesn't make sense in Java, where we return a List (we can't return an array due to the way Java generics work). It's also missing in Python." In this patch, I deprecated the method and changed the source files using it by replacing toArray with collect() directly Author: CodingCat Closes #105 from CodingCat/SPARK-1060 and squashes the following commits: 286f163 [CodingCat] deprecate in JavaRDDLike ee17b4e [CodingCat] add message and since 2ff7319 [CodingCat] deprecate toArray in RDD --- .../main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 1 + .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 1 + core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala | 4 ++-- .../src/main/scala/org/apache/spark/examples/SparkALS.scala | 4 ++-- .../scala/org/apache/spark/examples/mllib/SparkSVD.scala | 2 +- .../src/main/scala/org/apache/spark/mllib/linalg/SVD.scala | 4 ++-- .../test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala | 6 +++--- 8 files changed, 13 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index a89419bbd10e7..3df68d4ce508d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -283,6 +283,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return an array that contains all of the elements in this RDD. */ + @deprecated("use collect", "1.0.0") def toArray(): JList[T] = collect() /** 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 2384c8f2b6fd4..b20ed99f89f34 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -423,7 +423,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Return the key-value pairs in this RDD to the master as a Map. */ def collectAsMap(): Map[K, V] = { - val data = self.toArray() + val data = self.collect() val map = new mutable.HashMap[K, V] map.sizeHint(data.length) data.foreach { case (k, v) => map.put(k, v) } 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 4afa7523dd802..b50c9963b9d2c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -658,6 +658,7 @@ abstract class RDD[T: ClassTag]( /** * Return an array that contains all of the elements in this RDD. */ + @deprecated("use collect", "1.0.0") def toArray(): Array[T] = collect() /** diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index b50307cfa49b7..4ceea557f569c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -26,13 +26,13 @@ import cern.jet.random.engine.DRand import org.apache.spark.{Partition, TaskContext} -@deprecated("Replaced by PartitionwiseSampledRDDPartition", "1.0") +@deprecated("Replaced by PartitionwiseSampledRDDPartition", "1.0.0") private[spark] class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable { override val index: Int = prev.index } -@deprecated("Replaced by PartitionwiseSampledRDD", "1.0") +@deprecated("Replaced by PartitionwiseSampledRDD", "1.0.0") class SampledRDD[T: ClassTag]( prev: RDD[T], withReplacement: Boolean, diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 17bafc2218a31..ce4b3c8451e00 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -128,11 +128,11 @@ object SparkALS { println("Iteration " + iter + ":") ms = sc.parallelize(0 until M, slices) .map(i => update(i, msb.value(i), usb.value, Rc.value)) - .toArray + .collect() msb = sc.broadcast(ms) // Re-broadcast ms because it was updated us = sc.parallelize(0 until U, slices) .map(i => update(i, usb.value(i), msb.value, algebra.transpose(Rc.value))) - .toArray + .collect() usb = sc.broadcast(us) // Re-broadcast us because it was updated println("RMSE = " + rmse(R, ms, us)) println() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala index 19676fcc1a2b0..ce2b133368e85 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala @@ -54,6 +54,6 @@ object SparkSVD { val s = decomposed.S.data val v = decomposed.V.data - println("singular values = " + s.toArray.mkString) + println("singular values = " + s.collect().mkString) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index 8803c4c1a07be..e4a26eeb07c60 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -109,7 +109,7 @@ object SVD { // Construct jblas A^T A locally val ata = DoubleMatrix.zeros(n, n) - for (entry <- emits.toArray) { + for (entry <- emits.collect()) { ata.put(entry._1._1, entry._1._2, entry._2) } @@ -178,7 +178,7 @@ object SVD { val s = decomposed.S.data val v = decomposed.V.data - println("Computed " + s.toArray.length + " singular values and vectors") + println("Computed " + s.collect().length + " singular values and vectors") u.saveAsTextFile(output_u) s.saveAsTextFile(output_s) v.saveAsTextFile(output_v) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala index 32f3f141cd652..a92386865a189 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala @@ -50,7 +50,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val m = matrix.m val n = matrix.n val ret = DoubleMatrix.zeros(m, n) - matrix.data.toArray.map(x => ret.put(x.i, x.j, x.mval)) + matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) ret } @@ -106,7 +106,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val u = decomposed.U val s = decomposed.S val v = decomposed.V - val retrank = s.data.toArray.length + val retrank = s.data.collect().length assert(retrank == 1, "rank returned not one") @@ -139,7 +139,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val u = decomposed.U val s = decomposed.S val v = decomposed.V - val retrank = s.data.toArray.length + val retrank = s.data.collect().length val densea = getDenseMatrix(a) val svd = Singular.sparseSVD(densea) From 31a704004f9b4ad34f92ae5c95ae6e90d0ab62c7 Mon Sep 17 00:00:00 2001 From: jianghan Date: Wed, 12 Mar 2014 19:46:12 -0700 Subject: [PATCH 09/23] Fix example bug: compile error Author: jianghan Closes #132 from pooorman/master and squashes the following commits: 54afbe0 [jianghan] Fix example bug: compile error --- .../main/java/org/apache/spark/examples/JavaLogQuery.java | 2 +- .../main/java/org/apache/spark/examples/JavaPageRank.java | 6 +++--- .../main/java/org/apache/spark/examples/JavaWordCount.java | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index a518fe2f27eb0..617e4a6d045e0 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -124,7 +124,7 @@ public Stats call(Stats stats, Stats stats2) { List, Stats>> output = counts.collect(); for (Tuple2 t : output) { - System.out.println(t._1 + "\t" + t._2); + System.out.println(t._1() + "\t" + t._2()); } System.exit(0); } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index e53925b50c2ce..eb70fb547564c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -90,8 +90,8 @@ public Double call(List rs) { @Override public Iterable> call(Tuple2, Double> s) { List> results = new ArrayList>(); - for (String n : s._1) { - results.add(new Tuple2(n, s._2 / s._1.size())); + for (String n : s._1()) { + results.add(new Tuple2(n, s._2() / s._1().size())); } return results; } @@ -109,7 +109,7 @@ public Double call(Double sum) { // Collects all URL ranks and dump them to console. List> output = ranks.collect(); for (Tuple2 tuple : output) { - System.out.println(tuple._1 + " has rank: " + tuple._2 + "."); + System.out.println(tuple._1() + " has rank: " + tuple._2() + "."); } System.exit(0); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index fa1b977ab19f1..3ae1d8f7ca938 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -65,7 +65,7 @@ public Integer call(Integer i1, Integer i2) { List> output = counts.collect(); for (Tuple2 tuple : output) { - System.out.println(tuple._1 + ": " + tuple._2); + System.out.println(tuple._1() + ": " + tuple._2()); } System.exit(0); } From 6bd2eaa4a5bcf811c5b85be27c5e50058b5d0c12 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 12 Mar 2014 19:49:18 -0700 Subject: [PATCH 10/23] hot fix for PR105 - change to Java annotation Author: CodingCat Closes #133 from CodingCat/SPARK-1160-2 and squashes the following commits: 6607155 [CodingCat] hot fix for PR105 - change to Java annotation --- .../src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 3df68d4ce508d..2ba4fb8c18b1d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -282,8 +282,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return an array that contains all of the elements in this RDD. + * @deprecated As of Spark 1.0.0, toArray() is deprecated, use {@link #collect()} instead */ - @deprecated("use collect", "1.0.0") + @Deprecated def toArray(): JList[T] = collect() /** From 4ea23db0efff2f39ac5b8f0bd1d9a6ffa3eceb0d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 12 Mar 2014 23:16:59 -0700 Subject: [PATCH 11/23] SPARK-1019: pyspark RDD take() throws an NPE Author: Patrick Wendell Closes #112 from pwendell/pyspark-take and squashes the following commits: daae80e [Patrick Wendell] SPARK-1019: pyspark RDD take() throws an NPE --- core/src/main/scala/org/apache/spark/TaskContext.scala | 3 ++- .../scala/org/apache/spark/api/python/PythonRDD.scala | 8 ++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index cae983ed4c652..be53ca2968cfb 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -46,6 +46,7 @@ class TaskContext( } def executeOnCompleteCallbacks() { - onCompleteCallbacks.foreach{_()} + // Process complete callbacks in the reverse order of registration + onCompleteCallbacks.reverse.foreach{_()} } } 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 e4d0285710e84..b67286a4e3b75 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 @@ -100,6 +100,14 @@ private[spark] class PythonRDD[T: ClassTag]( } }.start() + /* + * Partial fix for SPARK-1019: Attempts to stop reading the input stream since + * other completion callbacks might invalidate the input. Because interruption + * is not synchronous this still leaves a potential race where the interruption is + * processed only after the stream becomes invalid. + */ + context.addOnCompleteCallback(() => context.interrupted = true) + // Return an iterator that read lines from the process's stdout val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) val stdoutIterator = new Iterator[Array[Byte]] { From e4e8d8f395aea48f0cae00d7c381a863c48a2837 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 13 Mar 2014 00:43:19 -0700 Subject: [PATCH 12/23] [SPARK-1237, 1238] Improve the computation of YtY for implicit ALS Computing YtY can be implemented using BLAS's DSPR operations instead of generating y_i y_i^T and then combining them. The latter generates many k-by-k matrices. On the movielens data, this change improves the performance by 10-20%. The algorithm remains the same, verified by computing RMSE on the movielens data. To compare the results, I also added an option to set a random seed in ALS. JIRA: 1. https://spark-project.atlassian.net/browse/SPARK-1237 2. https://spark-project.atlassian.net/browse/SPARK-1238 Author: Xiangrui Meng Closes #131 from mengxr/als and squashes the following commits: ed00432 [Xiangrui Meng] minor changes d984623 [Xiangrui Meng] minor changes 2fc1641 [Xiangrui Meng] remove commented code 4c7cde2 [Xiangrui Meng] allow specifying a random seed in ALS 200bef0 [Xiangrui Meng] optimize computeYtY and updateBlock --- .../spark/mllib/recommendation/ALS.scala | 174 ++++++++++++------ .../spark/mllib/recommendation/ALSSuite.scala | 15 +- 2 files changed, 134 insertions(+), 55 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 8958040e36640..777d0db2d6653 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -89,10 +89,15 @@ case class Rating(val user: Int, val product: Int, val rating: Double) * indicated user * preferences rather than explicit ratings given to items. */ -class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var lambda: Double, - var implicitPrefs: Boolean, var alpha: Double) - extends Serializable with Logging -{ +class ALS private ( + var numBlocks: Int, + var rank: Int, + var iterations: Int, + var lambda: Double, + var implicitPrefs: Boolean, + var alpha: Double, + var seed: Long = System.nanoTime() + ) extends Serializable with Logging { def this() = this(-1, 10, 10, 0.01, false, 1.0) /** @@ -132,6 +137,12 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l this } + /** Sets a random seed to have deterministic results. */ + def setSeed(seed: Long): ALS = { + this.seed = seed + this + } + /** * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. * Returns a MatrixFactorizationModel with feature vectors for each user and product. @@ -155,7 +166,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l // Initialize user and product factors randomly, but use a deterministic seed for each // partition so that fault recovery works - val seedGen = new Random() + val seedGen = new Random(seed) val seed1 = seedGen.nextInt() val seed2 = seedGen.nextInt() // Hash an integer to propagate random bits at all positions, similar to java.util.HashTable @@ -210,21 +221,46 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l */ def computeYtY(factors: RDD[(Int, Array[Array[Double]])]) = { if (implicitPrefs) { - Option( - factors.flatMapValues { case factorArray => - factorArray.view.map { vector => - val x = new DoubleMatrix(vector) - x.mmul(x.transpose()) - } - }.reduceByKeyLocally((a, b) => a.addi(b)) - .values - .reduce((a, b) => a.addi(b)) - ) + val n = rank * (rank + 1) / 2 + val LYtY = factors.values.aggregate(new DoubleMatrix(n))( seqOp = (L, Y) => { + Y.foreach(y => dspr(1.0, new DoubleMatrix(y), L)) + L + }, combOp = (L1, L2) => { + L1.addi(L2) + }) + val YtY = new DoubleMatrix(rank, rank) + fillFullMatrix(LYtY, YtY) + Option(YtY) } else { None } } + /** + * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's DSPR. + * + * @param L the lower triangular part of the matrix packed in an array (row major) + */ + private def dspr(alpha: Double, x: DoubleMatrix, L: DoubleMatrix) = { + val n = x.length + var i = 0 + var j = 0 + var idx = 0 + var axi = 0.0 + val xd = x.data + val Ld = L.data + while (i < n) { + axi = alpha * xd(i) + j = 0 + while (j <= i) { + Ld(idx) += axi * xd(j) + j += 1 + idx += 1 + } + i += 1 + } + } + /** * Flatten out blocked user or product factors into an RDD of (id, factor vector) pairs */ @@ -376,7 +412,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l for (productBlock <- 0 until numBlocks) { for (p <- 0 until blockFactors(productBlock).length) { val x = new DoubleMatrix(blockFactors(productBlock)(p)) - fillXtX(x, tempXtX) + tempXtX.fill(0.0) + dspr(1.0, x, tempXtX) val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) for (i <- 0 until us.length) { implicitPrefs match { @@ -387,7 +424,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l // Extension to the original paper to handle rs(i) < 0. confidence is a function // of |rs(i)| instead so that it is never negative: val confidence = 1 + alpha * abs(rs(i)) - userXtX(us(i)).addi(tempXtX.mul(confidence - 1)) + SimpleBlas.axpy(confidence - 1.0, tempXtX, userXtX(us(i))) // For rs(i) < 0, the corresponding entry in P is 0 now, not 1 -- negative rs(i) // means we try to reconstruct 0. We add terms only where P = 1, so, term below // is now only added for rs(i) > 0: @@ -400,38 +437,19 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l } // Solve the least-squares problem for each user and return the new feature vectors - userXtX.zipWithIndex.map{ case (triangularXtX, index) => + Array.range(0, numUsers).map { index => // Compute the full XtX matrix from the lower-triangular part we got above - fillFullMatrix(triangularXtX, fullXtX) + fillFullMatrix(userXtX(index), fullXtX) // Add regularization (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda) // Solve the resulting matrix, which is symmetric and positive-definite implicitPrefs match { case false => Solve.solvePositive(fullXtX, userXy(index)).data - case true => Solve.solvePositive(fullXtX.add(YtY.value.get), userXy(index)).data + case true => Solve.solvePositive(fullXtX.addi(YtY.value.get), userXy(index)).data } } } - /** - * Set xtxDest to the lower-triangular part of x transpose * x. For efficiency in summing - * these matrices, we store xtxDest as only rank * (rank+1) / 2 values, namely the values - * at (0,0), (1,0), (1,1), (2,0), (2,1), (2,2), etc in that order. - */ - private def fillXtX(x: DoubleMatrix, xtxDest: DoubleMatrix) { - var i = 0 - var pos = 0 - while (i < x.length) { - var j = 0 - while (j <= i) { - xtxDest.data(pos) = x.data(i) * x.data(j) - pos += 1 - j += 1 - } - i += 1 - } - } - /** * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square * matrix that it represents, storing it into destMatrix. @@ -455,9 +473,10 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l /** - * Top-level methods for calling Alternating Least Squares (ALS) matrix factorizaton. + * Top-level methods for calling Alternating Least Squares (ALS) matrix factorization. */ object ALS { + /** * Train a matrix factorization model given an RDD of ratings given by users to some products, * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the @@ -470,15 +489,39 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) * @param blocks level of parallelism to split computation into + * @param seed random seed */ def train( ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, - blocks: Int) - : MatrixFactorizationModel = - { + blocks: Int, + seed: Long + ): MatrixFactorizationModel = { + new ALS(blocks, rank, iterations, lambda, false, 1.0, seed).run(ratings) + } + + /** + * Train a matrix factorization model given an RDD of ratings given by users to some products, + * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the + * product of two lower-rank matrices of a given rank (number of features). To solve for these + * features, we run a given number of iterations of ALS. This is done using a level of + * parallelism given by `blocks`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + * @param lambda regularization factor (recommended: 0.01) + * @param blocks level of parallelism to split computation into + */ + def train( + ratings: RDD[Rating], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int + ): MatrixFactorizationModel = { new ALS(blocks, rank, iterations, lambda, false, 1.0).run(ratings) } @@ -495,8 +538,7 @@ object ALS { * @param lambda regularization factor (recommended: 0.01) */ def train(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double) - : MatrixFactorizationModel = - { + : MatrixFactorizationModel = { train(ratings, rank, iterations, lambda, -1) } @@ -512,8 +554,7 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) */ def train(ratings: RDD[Rating], rank: Int, iterations: Int) - : MatrixFactorizationModel = - { + : MatrixFactorizationModel = { train(ratings, rank, iterations, 0.01, -1) } @@ -530,6 +571,7 @@ object ALS { * @param lambda regularization factor (recommended: 0.01) * @param blocks level of parallelism to split computation into * @param alpha confidence parameter (only applies when immplicitPrefs = true) + * @param seed random seed */ def trainImplicit( ratings: RDD[Rating], @@ -537,9 +579,34 @@ object ALS { iterations: Int, lambda: Double, blocks: Int, - alpha: Double) - : MatrixFactorizationModel = - { + alpha: Double, + seed: Long + ): MatrixFactorizationModel = { + new ALS(blocks, rank, iterations, lambda, true, alpha, seed).run(ratings) + } + + /** + * Train a matrix factorization model given an RDD of 'implicit preferences' given by users + * to some products, in the form of (userID, productID, preference) pairs. We approximate the + * ratings matrix as the product of two lower-rank matrices of a given rank (number of features). + * To solve for these features, we run a given number of iterations of ALS. This is done using + * a level of parallelism given by `blocks`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + * @param lambda regularization factor (recommended: 0.01) + * @param blocks level of parallelism to split computation into + * @param alpha confidence parameter (only applies when immplicitPrefs = true) + */ + def trainImplicit( + ratings: RDD[Rating], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int, + alpha: Double + ): MatrixFactorizationModel = { new ALS(blocks, rank, iterations, lambda, true, alpha).run(ratings) } @@ -555,8 +622,8 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) */ - def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, - alpha: Double): MatrixFactorizationModel = { + def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, alpha: Double) + : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, lambda, -1, alpha) } @@ -573,8 +640,7 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) */ def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int) - : MatrixFactorizationModel = - { + : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 45e7d2db00c42..5aab9aba8f9c0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -23,9 +23,10 @@ import scala.util.Random import org.scalatest.FunSuite -import org.jblas._ +import org.jblas.DoubleMatrix import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.SparkContext._ object ALSSuite { @@ -115,6 +116,18 @@ class ALSSuite extends FunSuite with LocalSparkContext { testALS(100, 200, 2, 15, 0.7, 0.4, true, false, true) } + test("pseudorandomness") { + val ratings = sc.parallelize(ALSSuite.generateRatings(10, 20, 5, 0.5, false, false)._1, 2) + val model11 = ALS.train(ratings, 5, 1, 1.0, 2, 1) + val model12 = ALS.train(ratings, 5, 1, 1.0, 2, 1) + val u11 = model11.userFeatures.values.flatMap(_.toList).collect().toList + val u12 = model12.userFeatures.values.flatMap(_.toList).collect().toList + val model2 = ALS.train(ratings, 5, 1, 1.0, 2, 2) + val u2 = model2.userFeatures.values.flatMap(_.toList).collect().toList + assert(u11 == u12) + assert(u11 != u2) + } + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * From 698373211ef3cdf841c82d48168cd5dbe00a57b4 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 13 Mar 2014 12:11:33 -0700 Subject: [PATCH 13/23] SPARK-1183. Don't use "worker" to mean executor Author: Sandy Ryza Closes #120 from sryza/sandy-spark-1183 and squashes the following commits: 5066a4a [Sandy Ryza] Remove "worker" in a couple comments 0bd1e46 [Sandy Ryza] Remove --am-class from usage bfc8fe0 [Sandy Ryza] Remove am-class from doc and fix yarn-alpha 607539f [Sandy Ryza] Address review comments 74d087a [Sandy Ryza] SPARK-1183. Don't use "worker" to mean executor --- docs/cluster-overview.md | 2 +- docs/graphx-programming-guide.md | 2 +- docs/job-scheduling.md | 4 +- docs/mllib-classification-regression.md | 4 +- docs/python-programming-guide.md | 6 +- docs/running-on-yarn.md | 29 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 38 ++--- ...rLauncher.scala => ExecutorLauncher.scala} | 28 ++-- ...rRunnable.scala => ExecutorRunnable.scala} | 14 +- .../deploy/yarn/YarnAllocationHandler.scala | 124 ++++++++-------- .../yarn/ApplicationMasterArguments.scala | 27 ++-- .../spark/deploy/yarn/ClientArguments.scala | 46 ++++-- .../apache/spark/deploy/yarn/ClientBase.scala | 18 +-- .../yarn/ClientDistributedCacheManager.scala | 4 +- ...eUtil.scala => ExecutorRunnableUtil.scala} | 14 +- .../cluster/YarnClientClusterScheduler.scala | 4 +- .../cluster/YarnClientSchedulerBackend.scala | 26 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 38 ++--- ...rLauncher.scala => ExecutorLauncher.scala} | 26 ++-- ...rRunnable.scala => ExecutorRunnable.scala} | 14 +- .../deploy/yarn/YarnAllocationHandler.scala | 138 +++++++++--------- 21 files changed, 312 insertions(+), 294 deletions(-) rename yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/{WorkerLauncher.scala => ExecutorLauncher.scala} (91%) rename yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/{WorkerRunnable.scala => ExecutorRunnable.scala} (93%) rename yarn/common/src/main/scala/org/apache/spark/deploy/yarn/{WorkerRunnableUtil.scala => ExecutorRunnableUtil.scala} (95%) rename yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/{WorkerLauncher.scala => ExecutorLauncher.scala} (92%) rename yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/{WorkerRunnable.scala => ExecutorRunnable.scala} (90%) diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index e16703292cc22..a555a7b5023e3 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -13,7 +13,7 @@ object in your main program (called the _driver program_). Specifically, to run on a cluster, the SparkContext can connect to several types of _cluster managers_ (either Spark's own standalone cluster manager or Mesos/YARN), which allocate resources across applications. Once connected, Spark acquires *executors* on nodes in the cluster, which are -worker processes that run computations and store data for your application. +processes that run computations and store data for your application. Next, it sends your application code (defined by JAR or Python files passed to SparkContext) to the executors. Finally, SparkContext sends *tasks* for the executors to run. diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 3dfed7bea9ea8..1238e3e0a4e7d 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -135,7 +135,7 @@ Like RDDs, property graphs are immutable, distributed, and fault-tolerant. Chan structure of the graph are accomplished by producing a new graph with the desired changes. Note that substantial parts of the original graph (i.e., unaffected structure, attributes, and indicies) are reused in the new graph reducing the cost of this inherently functional data-structure. The -graph is partitioned across the workers using a range of vertex-partitioning heuristics. As with +graph is partitioned across the executors using a range of vertex-partitioning heuristics. As with RDDs, each partition of the graph can be recreated on a different machine in the event of a failure. Logically the property graph corresponds to a pair of typed collections (RDDs) encoding the diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index df2faa5e41b18..94604f301dd46 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -39,8 +39,8 @@ Resource allocation can be configured as follows, based on the cluster type: * **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`, and optionally set `spark.cores.max` to limit each application's resource share as in the standalone mode. You should also set `spark.executor.memory` to control the executor memory. -* **YARN:** The `--num-workers` option to the Spark YARN client controls how many workers it will allocate - on the cluster, while `--worker-memory` and `--worker-cores` control the resources per worker. +* **YARN:** The `--num-executors` option to the Spark YARN client controls how many executors it will allocate + on the cluster, while `--executor-memory` and `--executor-cores` control the resources per executor. A second option available on Mesos is _dynamic sharing_ of CPU cores. In this mode, each Spark application still has a fixed and independent memory allocation (set by `spark.executor.memory`), but when the diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index 18a3e8e075086..d5bd8042ca2ec 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -77,8 +77,8 @@ between the two goals of small loss and small model complexity. **Distributed Datasets.** For all currently implemented optimization methods for classification, the data must be -distributed between the worker machines *by examples*. Every machine holds a consecutive block of -the `$n$` example/label pairs `$(\x_i,y_i)$`. +distributed between processes on the worker machines *by examples*. Machines hold consecutive +blocks of the `$n$` example/label pairs `$(\x_i,y_i)$`. In other words, the input distributed dataset ([RDD](scala-programming-guide.html#resilient-distributed-datasets-rdds)) must be the set of vectors `$\x_i\in\R^d$`. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 57ed54c9cf4c0..cbe7d820b455e 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -43,9 +43,9 @@ def is_error(line): errors = logData.filter(is_error) {% endhighlight %} -PySpark will automatically ship these functions to workers, along with any objects that they reference. -Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers. -The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers. +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. In addition, PySpark fully supports interactive use---simply run `./bin/pyspark` to launch an interactive shell. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b17929542c531..2e9dec4856ee9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -41,7 +41,7 @@ System Properties: * `spark.yarn.submit.file.replication`, the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives. * `spark.yarn.preserve.staging.files`, set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them. * `spark.yarn.scheduler.heartbeat.interval-ms`, the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds. -* `spark.yarn.max.worker.failures`, the maximum number of executor failures before failing the application. Default is the number of executors requested times 2 with minimum of 3. +* `spark.yarn.max.executor.failures`, the maximum number of executor failures before failing the application. Default is the number of executors requested times 2 with minimum of 3. # Launching Spark on YARN @@ -60,11 +60,10 @@ The command to launch the Spark application on the cluster is as follows: --jar \ --class \ --args \ - --num-workers \ - --master-class - --master-memory \ - --worker-memory \ - --worker-cores \ + --num-executors \ + --driver-memory \ + --executor-memory \ + --executor-cores \ --name \ --queue \ --addJars \ @@ -85,10 +84,10 @@ For example: --jar examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ --class org.apache.spark.examples.SparkPi \ --args yarn-cluster \ - --num-workers 3 \ - --master-memory 4g \ - --worker-memory 2g \ - --worker-cores 1 + --num-executors 3 \ + --driver-memory 4g \ + --executor-memory 2g \ + --executor-cores 1 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. @@ -100,12 +99,12 @@ With yarn-client mode, the application will be launched locally, just like runni Configuration in yarn-client mode: -In order to tune worker cores/number/memory etc., you need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. +In order to tune executor cores/number/memory etc., you need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. -* `SPARK_WORKER_INSTANCES`, Number of executors to start (Default: 2) -* `SPARK_WORKER_CORES`, Number of cores per executor (Default: 1). -* `SPARK_WORKER_MEMORY`, Memory per executor (e.g. 1000M, 2G) (Default: 1G) -* `SPARK_MASTER_MEMORY`, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) +* `SPARK_EXECUTOR_INSTANCES`, Number of executors to start (Default: 2) +* `SPARK_EXECUTOR_CORES`, Number of cores per executor (Default: 1). +* `SPARK_EXECUTOR_MEMORY`, Memory per executor (e.g. 1000M, 2G) (Default: 1G) +* `SPARK_DRIVER_MEMORY`, Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb) * `SPARK_YARN_APP_NAME`, The name of your application (Default: Spark) * `SPARK_YARN_QUEUE`, The YARN queue to use for allocation requests (Default: 'default') * `SPARK_YARN_DIST_FILES`, Comma separated list of files to be distributed with the job. diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 87785cdc60c52..910484ed5432a 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -61,9 +61,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true - // Default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", - math.max(args.numWorkers * 2, 3)) + // Default to numExecutors * 2, with minimum of 3 + private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", + sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) private var registered = false @@ -96,7 +96,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // Call this to force generation of secret so it gets populated into the // hadoop UGI. This has to happen before the startUserClass which does a - // doAs in order for the credentials to be passed on to the worker containers. + // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) // Start the user's JAR @@ -115,7 +115,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } // Allocate all containers - allocateWorkers() + allocateExecutors() // Wait for the user class to Finish userThread.join() @@ -215,7 +215,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, t } - // this need to happen before allocateWorkers + // this need to happen before allocateExecutors private def waitForSparkContextInitialized() { logInfo("Waiting for spark context initialization") try { @@ -260,21 +260,21 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } } - private def allocateWorkers() { + private def allocateExecutors() { try { - logInfo("Allocating " + args.numWorkers + " workers.") + logInfo("Allocating " + args.numExecutors + " executors.") // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure // Exists the loop if the user thread exits. - while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") + "max number of executor failures reached") } yarnAllocator.allocateContainers( - math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) ApplicationMaster.incrementAllocatorLoop(1) Thread.sleep(100) } @@ -283,7 +283,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } - logInfo("All workers have launched.") + logInfo("All executors have launched.") // Launch a progress reporter thread, else the app will get killed after expiration // (def: 10mins) timeout. @@ -309,15 +309,15 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, val t = new Thread { override def run() { while (userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") + "max number of executor failures reached") } - val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - if (missingWorkerCount > 0) { + val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning + if (missingExecutorCount > 0) { logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingWorkerCount)) - yarnAllocator.allocateContainers(missingWorkerCount) + format(missingExecutorCount)) + yarnAllocator.allocateContainers(missingExecutorCount) } else sendProgress() Thread.sleep(sleepTime) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala similarity index 91% rename from yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index b735d01df8097..7b0e020263835 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo -class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) +class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) @@ -89,7 +89,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() if (minimumMemory > 0) { - val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD + val mem = args.executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) if (numCore > 0) { @@ -102,7 +102,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar waitForSparkMaster() // Allocate all containers - allocateWorkers() + allocateExecutors() // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. @@ -199,7 +199,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar } - private def allocateWorkers() { + private def allocateExecutors() { // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = @@ -208,16 +208,16 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, preferredNodeLocationData, sparkConf) - logInfo("Allocating " + args.numWorkers + " workers.") + logInfo("Allocating " + args.numExecutors + " executors.") // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - while ((yarnAllocator.getNumWorkersRunning < args.numWorkers) && (!driverClosed)) { - yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { + yarnAllocator.allocateContainers(math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) Thread.sleep(100) } - logInfo("All workers have launched.") + logInfo("All executors have launched.") } @@ -228,10 +228,10 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar val t = new Thread { override def run() { while (!driverClosed) { - val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - if (missingWorkerCount > 0) { - logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") - yarnAllocator.allocateContainers(missingWorkerCount) + val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning + if (missingExecutorCount > 0) { + logInfo("Allocating " + missingExecutorCount + " containers to make up for (potentially ?) lost containers") + yarnAllocator.allocateContainers(missingExecutorCount) } else sendProgress() Thread.sleep(sleepTime) @@ -264,9 +264,9 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar } -object WorkerLauncher { +object ExecutorLauncher { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new WorkerLauncher(args).run() + new ExecutorLauncher(args).run() } } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala similarity index 93% rename from yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 8c686e393f4f8..981e8b05f602d 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -38,16 +38,16 @@ import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} import org.apache.spark.{SparkConf, Logging} -class WorkerRunnable( +class ExecutorRunnable( container: Container, conf: Configuration, spConf: SparkConf, masterAddress: String, slaveId: String, hostname: String, - workerMemory: Int, - workerCores: Int) - extends Runnable with WorkerRunnableUtil with Logging { + executorMemory: Int, + executorCores: Int) + extends Runnable with ExecutorRunnableUtil with Logging { var rpc: YarnRPC = YarnRPC.create(conf) var cm: ContainerManager = _ @@ -55,7 +55,7 @@ class WorkerRunnable( val yarnConf: YarnConfiguration = new YarnConfiguration(conf) def run = { - logInfo("Starting Worker Container") + logInfo("Starting Executor Container") cm = connectToCM startContainer } @@ -81,8 +81,8 @@ class WorkerRunnable( credentials.writeTokenStorageToStream(dob) ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, workerMemory, workerCores) - logInfo("Setting up worker with commands: " + commands) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) + logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) // Send the start request to the ContainerManager diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index e91257be8ed00..2056667af50cb 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -58,9 +58,9 @@ private[yarn] class YarnAllocationHandler( val conf: Configuration, val resourceManager: AMRMProtocol, val appAttemptId: ApplicationAttemptId, - val maxWorkers: Int, - val workerMemory: Int, - val workerCores: Int, + val maxExecutors: Int, + val executorMemory: Int, + val executorCores: Int, val preferredHostToCount: Map[String, Int], val preferredRackToCount: Map[String, Int], val sparkConf: SparkConf) @@ -84,39 +84,39 @@ private[yarn] class YarnAllocationHandler( // Containers to be released in next request to RM private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] - private val numWorkersRunning = new AtomicInteger() - // Used to generate a unique id per worker - private val workerIdCounter = new AtomicInteger() + private val numExecutorsRunning = new AtomicInteger() + // Used to generate a unique id per executor + private val executorIdCounter = new AtomicInteger() private val lastResponseId = new AtomicInteger() - private val numWorkersFailed = new AtomicInteger() + private val numExecutorsFailed = new AtomicInteger() - def getNumWorkersRunning: Int = numWorkersRunning.intValue + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue - def getNumWorkersFailed: Int = numWorkersFailed.intValue + def getNumExecutorsFailed: Int = numExecutorsFailed.intValue def isResourceConstraintSatisfied(container: Container): Boolean = { - container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + container.getResource.getMemory >= (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD) } - def allocateContainers(workersToRequest: Int) { + def allocateContainers(executorsToRequest: Int) { // We need to send the request only once from what I understand ... but for now, not modifying // this much. // Keep polling the Resource Manager for containers - val amResp = allocateWorkerResources(workersToRequest).getAMResponse + val amResp = allocateExecutorResources(executorsToRequest).getAMResponse val _allocatedContainers = amResp.getAllocatedContainers() if (_allocatedContainers.size > 0) { logDebug(""" Allocated containers: %d - Current worker count: %d + Current executor count: %d Containers released: %s Containers to be released: %s Cluster resources: %s """.format( _allocatedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers, amResp.getAvailableResources)) @@ -221,59 +221,59 @@ private[yarn] class YarnAllocationHandler( // Run each of the allocated containers for (container <- allocatedContainers) { - val numWorkersRunningNow = numWorkersRunning.incrementAndGet() - val workerHostname = container.getNodeId.getHost + val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() + val executorHostname = container.getNodeId.getHost val containerId = container.getId assert( - container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) + container.getResource.getMemory >= (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) - if (numWorkersRunningNow > maxWorkers) { + if (numExecutorsRunningNow > maxExecutors) { logInfo("""Ignoring container %s at host %s, since we already have the required number of - containers for it.""".format(containerId, workerHostname)) + containers for it.""".format(containerId, executorHostname)) releasedContainerList.add(containerId) // reset counter back to old value. - numWorkersRunning.decrementAndGet() + numExecutorsRunning.decrementAndGet() } else { // Deallocate + allocate can result in reusing id's wrongly - so use a different counter - // (workerIdCounter) - val workerId = workerIdCounter.incrementAndGet().toString + // (executorIdCounter) + val executorId = executorIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - logInfo("launching container on " + containerId + " host " + workerHostname) + logInfo("launching container on " + containerId + " host " + executorHostname) // Just to be safe, simply remove it from pendingReleaseContainers. // Should not be there, but .. pendingReleaseContainers.remove(containerId) - val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) + val rack = YarnAllocationHandler.lookupRack(conf, executorHostname) allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]()) containerSet += containerId - allocatedContainerToHostMap.put(containerId, workerHostname) + allocatedContainerToHostMap.put(containerId, executorHostname) if (rack != null) { allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) } } new Thread( - new WorkerRunnable(container, conf, sparkConf, driverUrl, workerId, - workerHostname, workerMemory, workerCores) + new ExecutorRunnable(container, conf, sparkConf, driverUrl, executorId, + executorHostname, executorMemory, executorCores) ).start() } } logDebug(""" Finished processing %d containers. - Current number of workers running: %d, + Current number of executors running: %d, releasedContainerList: %s, pendingReleaseContainers: %s """.format( allocatedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers)) } @@ -292,7 +292,7 @@ private[yarn] class YarnAllocationHandler( } else { // Simply decrement count - next iteration of ReporterThread will take care of allocating. - numWorkersRunning.decrementAndGet() + numExecutorsRunning.decrementAndGet() logInfo("Completed container %s (state: %s, exit status: %s)".format( containerId, completedContainer.getState, @@ -302,7 +302,7 @@ private[yarn] class YarnAllocationHandler( // now I think its ok as none of the containers are expected to exit if (completedContainer.getExitStatus() != 0) { logInfo("Container marked as failed: " + containerId) - numWorkersFailed.incrementAndGet() + numExecutorsFailed.incrementAndGet() } } @@ -332,12 +332,12 @@ private[yarn] class YarnAllocationHandler( } logDebug(""" Finished processing %d completed containers. - Current number of workers running: %d, + Current number of executors running: %d, releasedContainerList: %s, pendingReleaseContainers: %s """.format( completedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers)) } @@ -387,18 +387,18 @@ private[yarn] class YarnAllocationHandler( retval } - private def allocateWorkerResources(numWorkers: Int): AllocateResponse = { + private def allocateExecutorResources(numExecutors: Int): AllocateResponse = { var resourceRequests: List[ResourceRequest] = null // default. - if (numWorkers <= 0 || preferredHostToCount.isEmpty) { - logDebug("numWorkers: " + numWorkers + ", host preferences: " + preferredHostToCount.isEmpty) + if (numExecutors <= 0 || preferredHostToCount.isEmpty) { + logDebug("numExecutors: " + numExecutors + ", host preferences: " + preferredHostToCount.isEmpty) resourceRequests = List( - createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)) + createResourceRequest(AllocationType.ANY, null, numExecutors, YarnAllocationHandler.PRIORITY)) } else { - // request for all hosts in preferred nodes and for numWorkers - + // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. val hostContainerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](preferredHostToCount.size) @@ -419,7 +419,7 @@ private[yarn] class YarnAllocationHandler( val anyContainerRequests: ResourceRequest = createResourceRequest( AllocationType.ANY, resource = null, - numWorkers, + numExecutors, YarnAllocationHandler.PRIORITY) val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( @@ -441,9 +441,9 @@ private[yarn] class YarnAllocationHandler( val releasedContainerList = createReleasedContainerList() req.addAllReleases(releasedContainerList) - if (numWorkers > 0) { - logInfo("Allocating %d worker containers with %d of memory each.".format(numWorkers, - workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) + if (numExecutors > 0) { + logInfo("Allocating %d executor containers with %d of memory each.".format(numExecutors, + executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) } else { logDebug("Empty allocation req .. release : " + releasedContainerList) @@ -464,7 +464,7 @@ private[yarn] class YarnAllocationHandler( private def createResourceRequest( requestType: AllocationType.AllocationType, resource:String, - numWorkers: Int, + numExecutors: Int, priority: Int): ResourceRequest = { // If hostname specified, we need atleast two requests - node local and rack local. @@ -473,7 +473,7 @@ private[yarn] class YarnAllocationHandler( case AllocationType.HOST => { assert(YarnAllocationHandler.ANY_HOST != resource) val hostname = resource - val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority) + val nodeLocal = createResourceRequestImpl(hostname, numExecutors, priority) // Add to host->rack mapping YarnAllocationHandler.populateRackInfo(conf, hostname) @@ -482,10 +482,10 @@ private[yarn] class YarnAllocationHandler( } case AllocationType.RACK => { val rack = resource - createResourceRequestImpl(rack, numWorkers, priority) + createResourceRequestImpl(rack, numExecutors, priority) } case AllocationType.ANY => createResourceRequestImpl( - YarnAllocationHandler.ANY_HOST, numWorkers, priority) + YarnAllocationHandler.ANY_HOST, numExecutors, priority) case _ => throw new IllegalArgumentException( "Unexpected/unsupported request type: " + requestType) } @@ -493,13 +493,13 @@ private[yarn] class YarnAllocationHandler( private def createResourceRequestImpl( hostname:String, - numWorkers: Int, + numExecutors: Int, priority: Int): ResourceRequest = { val rsrcRequest = Records.newRecord(classOf[ResourceRequest]) val memCapability = Records.newRecord(classOf[Resource]) // There probably is some overhead here, let's reserve a bit more memory. - memCapability.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + memCapability.setMemory(executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD) rsrcRequest.setCapability(memCapability) val pri = Records.newRecord(classOf[Priority]) @@ -508,7 +508,7 @@ private[yarn] class YarnAllocationHandler( rsrcRequest.setHostName(hostname) - rsrcRequest.setNumContainers(java.lang.Math.max(numWorkers, 0)) + rsrcRequest.setNumContainers(java.lang.Math.max(numExecutors, 0)) rsrcRequest } @@ -560,9 +560,9 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numWorkers, - args.workerMemory, - args.workerCores, + args.numExecutors, + args.executorMemory, + args.executorCores, Map[String, Int](), Map[String, Int](), sparkConf) @@ -582,9 +582,9 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numWorkers, - args.workerMemory, - args.workerCores, + args.numExecutors, + args.executorMemory, + args.executorCores, hostToCount, rackToCount, sparkConf) @@ -594,9 +594,9 @@ object YarnAllocationHandler { conf: Configuration, resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, - maxWorkers: Int, - workerMemory: Int, - workerCores: Int, + maxExecutors: Int, + executorMemory: Int, + executorCores: Int, map: collection.Map[String, collection.Set[SplitInfo]], sparkConf: SparkConf): YarnAllocationHandler = { @@ -606,9 +606,9 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - maxWorkers, - workerMemory, - workerCores, + maxExecutors, + executorMemory, + executorCores, hostToCount, rackToCount, sparkConf) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index f76a5ddd39e90..25cc9016b10a6 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -24,9 +24,9 @@ class ApplicationMasterArguments(val args: Array[String]) { var userJar: String = null var userClass: String = null var userArgs: Seq[String] = Seq[String]() - var workerMemory = 1024 - var workerCores = 1 - var numWorkers = 2 + var executorMemory = 1024 + var executorCores = 1 + var numExecutors = 2 parseArgs(args.toList) @@ -36,7 +36,8 @@ class ApplicationMasterArguments(val args: Array[String]) { var args = inputArgs while (! args.isEmpty) { - + // --num-workers, --worker-memory, and --worker-cores are deprecated since 1.0, + // the properties with executor in their names are preferred. args match { case ("--jar") :: value :: tail => userJar = value @@ -50,16 +51,16 @@ class ApplicationMasterArguments(val args: Array[String]) { userArgsBuffer += value args = tail - case ("--num-workers") :: IntParam(value) :: tail => - numWorkers = value + case ("--num-workers" | "--num-executors") :: IntParam(value) :: tail => + numExecutors = value args = tail - case ("--worker-memory") :: IntParam(value) :: tail => - workerMemory = value + case ("--worker-memory" | "--executor-memory") :: IntParam(value) :: tail => + executorMemory = value args = tail - case ("--worker-cores") :: IntParam(value) :: tail => - workerCores = value + case ("--worker-cores" | "--executor-cores") :: IntParam(value) :: tail => + executorCores = value args = tail case Nil => @@ -86,9 +87,9 @@ class ApplicationMasterArguments(val args: Array[String]) { " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + - " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + - " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n") + " --num-executors NUM Number of executors to start (Default: 2)\n" + + " --executor-cores NUM Number of cores for the executors (Default: 1)\n" + + " --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G)\n") System.exit(exitCode) } } 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 1f894a677d169..a001060cdb746 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 @@ -33,9 +33,9 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var userJar: String = null var userClass: String = null var userArgs: Seq[String] = Seq[String]() - var workerMemory = 1024 // MB - var workerCores = 1 - var numWorkers = 2 + var executorMemory = 1024 // MB + var executorCores = 1 + var numExecutors = 2 var amQueue = sparkConf.get("QUEUE", "default") var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" @@ -67,24 +67,39 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { userArgsBuffer += value args = tail - case ("--master-class") :: value :: tail => + case ("--master-class" | "--am-class") :: value :: tail => + if (args(0) == "--master-class") { + println("--master-class is deprecated. Use --am-class instead.") + } amClass = value args = tail - case ("--master-memory") :: MemoryParam(value) :: tail => + case ("--master-memory" | "--driver-memory") :: MemoryParam(value) :: tail => + if (args(0) == "--master-memory") { + println("--master-memory is deprecated. Use --driver-memory instead.") + } amMemory = value args = tail - case ("--num-workers") :: IntParam(value) :: tail => - numWorkers = value + case ("--num-workers" | "--num-executors") :: IntParam(value) :: tail => + if (args(0) == "--num-workers") { + println("--num-workers is deprecated. Use --num-executors instead.") + } + numExecutors = value args = tail - case ("--worker-memory") :: MemoryParam(value) :: tail => - workerMemory = value + case ("--worker-memory" | "--executor-memory") :: MemoryParam(value) :: tail => + if (args(0) == "--worker-memory") { + println("--worker-memory is deprecated. Use --executor-memory instead.") + } + executorMemory = value args = tail - case ("--worker-cores") :: IntParam(value) :: tail => - workerCores = value + case ("--worker-cores" | "--executor-memory") :: IntParam(value) :: tail => + if (args(0) == "--worker-cores") { + println("--worker-cores is deprecated. Use --executor-cores instead.") + } + executorCores = value args = tail case ("--queue") :: value :: tail => @@ -133,11 +148,10 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + - " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1).\n" + - " --master-class CLASS_NAME Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" + - " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + - " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + + " --num-executors NUM Number of executors to start (Default: 2)\n" + + " --executor-cores NUM Number of cores for the executors (Default: 1).\n" + + " --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb)\n" + + " --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G)\n" + " --name NAME The name of your application (Default: Spark)\n" + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 74c5e0f18e7bd..57e5761cba896 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -73,10 +73,10 @@ trait ClientBase extends Logging { ((args.userJar == null && args.amClass == classOf[ApplicationMaster].getName) -> "Error: You must specify a user jar when running in standalone mode!"), (args.userClass == null) -> "Error: You must specify a user class!", - (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!", + (args.numExecutors <= 0) -> "Error: You must specify at least 1 executor!", (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" + "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), - (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" + + (args.executorMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Executor memory size" + "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString) ).foreach { case(cond, errStr) => if (cond) { @@ -95,9 +95,9 @@ trait ClientBase extends Logging { logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) // If we have requested more then the clusters max for a single resource then exit. - if (args.workerMemory > maxMem) { - logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.". - format(args.workerMemory, maxMem)) + if (args.executorMemory > maxMem) { + logError("Required executor memory (%d MB), is above the max threshold (%d MB) of this cluster.". + format(args.executorMemory, maxMem)) System.exit(1) } val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD @@ -276,7 +276,7 @@ trait ClientBase extends Logging { env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() - // Set the environment variables to be passed on to the Workers. + // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) @@ -360,9 +360,9 @@ trait ClientBase extends Logging { " --class " + args.userClass + " --jar " + args.userJar + userArgsToString(args) + - " --worker-memory " + args.workerMemory + - " --worker-cores " + args.workerCores + - " --num-workers " + args.numWorkers + + " --executor-memory " + args.executorMemory + + " --executor-cores " + args.executorCores + + " --num-executors " + args.numExecutors + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 535abbfb7f638..68cda0f1c9f8b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -46,10 +46,10 @@ class ClientDistributedCacheManager() extends Logging { /** * Add a resource to the list of distributed cache resources. This list can - * be sent to the ApplicationMaster and possibly the workers so that it can + * be sent to the ApplicationMaster and possibly the executors so that it can * be downloaded into the Hadoop distributed cache for use by this application. * Adds the LocalResource to the localResources HashMap passed in and saves - * the stats of the resources to they can be sent to the workers and verified. + * the stats of the resources to they can be sent to the executors and verified. * * @param fs FileSystem * @param conf Configuration diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala similarity index 95% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index bfa8f84bf7f85..da0a6f74efcd5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -39,7 +39,7 @@ import org.apache.spark.{SparkConf, Logging} import org.apache.hadoop.yarn.conf.YarnConfiguration -trait WorkerRunnableUtil extends Logging { +trait ExecutorRunnableUtil extends Logging { val yarnConf: YarnConfiguration val sparkConf: SparkConf @@ -49,13 +49,13 @@ trait WorkerRunnableUtil extends Logging { masterAddress: String, slaveId: String, hostname: String, - workerMemory: Int, - workerCores: Int) = { + executorMemory: Int, + executorCores: Int) = { // Extra options for the JVM var JAVA_OPTS = "" // Set the JVM memory - val workerMemoryString = workerMemory + "m" - JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " " + val executorMemoryString = executorMemory + "m" + JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " if (env.isDefinedAt("SPARK_JAVA_OPTS")) { JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " } @@ -97,7 +97,7 @@ trait WorkerRunnableUtil extends Logging { val commands = List[String](javaCommand + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. - // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in + // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in // an inconsistent state. // TODO: If the OOM is not recoverable by rescheduling it on different node, then do // 'something' to fail job ... akin to blacklisting trackers in mapred ? @@ -107,7 +107,7 @@ trait WorkerRunnableUtil extends Logging { masterAddress + " " + slaveId + " " + hostname + " " + - workerCores + + executorCores + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala index 522e0a9ad7eeb..6b91e6b9eb899 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -25,7 +25,7 @@ import org.apache.spark.util.Utils /** * - * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM. + * This scheduler launches executors through Yarn - by calling into Client to launch ExecutorLauncher as AM. */ private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) { @@ -40,7 +40,7 @@ private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configur override def postStartHook() { - // The yarn application is running, but the worker might not yet ready + // The yarn application is running, but the executor might not yet ready // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt Thread.sleep(2000L) logInfo("YarnClientClusterScheduler.postStartHook done") diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index e7130d24072ca..d1f13e3c369ed 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -53,20 +53,24 @@ private[spark] class YarnClientSchedulerBackend( "--class", "notused", "--jar", null, "--args", hostport, - "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" + "--am-class", "org.apache.spark.deploy.yarn.ExecutorLauncher" ) // process any optional arguments, use the defaults already defined in ClientArguments // if things aren't specified - Map("--master-memory" -> "SPARK_MASTER_MEMORY", - "--num-workers" -> "SPARK_WORKER_INSTANCES", - "--worker-memory" -> "SPARK_WORKER_MEMORY", - "--worker-cores" -> "SPARK_WORKER_CORES", - "--queue" -> "SPARK_YARN_QUEUE", - "--name" -> "SPARK_YARN_APP_NAME", - "--files" -> "SPARK_YARN_DIST_FILES", - "--archives" -> "SPARK_YARN_DIST_ARCHIVES") - .foreach { case (optName, optParam) => addArg(optName, optParam, argsArrayBuf) } + Map("SPARK_MASTER_MEMORY" -> "--driver-memory", + "SPARK_DRIVER_MEMORY" -> "--driver-memory", + "SPARK_WORKER_INSTANCES" -> "--num-executors", + "SPARK_WORKER_MEMORY" -> "--executor-memory", + "SPARK_WORKER_CORES" -> "--executor-cores", + "SPARK_EXECUTOR_INSTANCES" -> "--num-executors", + "SPARK_EXECUTOR_MEMORY" -> "--executor-memory", + "SPARK_EXECUTOR_CORES" -> "--executor-cores", + "SPARK_YARN_QUEUE" -> "--queue", + "SPARK_YARN_APP_NAME" -> "--name", + "SPARK_YARN_DIST_FILES" -> "--files", + "SPARK_YARN_DIST_ARCHIVES" -> "--archives") + .foreach { case (optParam, optName) => addArg(optName, optParam, argsArrayBuf) } logDebug("ClientArguments called with: " + argsArrayBuf) val args = new ClientArguments(argsArrayBuf.toArray, conf) @@ -77,7 +81,7 @@ private[spark] class YarnClientSchedulerBackend( def waitForApp() { - // TODO : need a better way to find out whether the workers are ready or not + // TODO : need a better way to find out whether the executors are ready or not // maybe by resource usage report? while(true) { val report = client.getApplicationReport(appId) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 57d15774290dd..30735cbfdf26e 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -64,9 +64,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var isLastAMRetry: Boolean = true private var amClient: AMRMClient[ContainerRequest] = _ - // Default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", - math.max(args.numWorkers * 2, 3)) + // Default to numExecutors * 2, with minimum of 3 + private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", + sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) private var registered = false @@ -101,7 +101,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // Call this to force generation of secret so it gets populated into the // hadoop UGI. This has to happen before the startUserClass which does a - // doAs in order for the credentials to be passed on to the worker containers. + // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) // Start the user's JAR @@ -120,7 +120,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } // Allocate all containers - allocateWorkers() + allocateExecutors() // Wait for the user class to Finish userThread.join() @@ -202,7 +202,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, t } - // This need to happen before allocateWorkers() + // This need to happen before allocateExecutors() private def waitForSparkContextInitialized() { logInfo("Waiting for Spark context initialization") try { @@ -247,18 +247,18 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } } - private def allocateWorkers() { + private def allocateExecutors() { try { - logInfo("Allocating " + args.numWorkers + " workers.") + logInfo("Allocating " + args.numExecutors + " executors.") // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - yarnAllocator.addResourceRequests(args.numWorkers) + yarnAllocator.addResourceRequests(args.numExecutors) // Exits the loop if the user thread exits. - while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") + "max number of executor failures reached") } yarnAllocator.allocateResources() ApplicationMaster.incrementAllocatorLoop(1) @@ -269,7 +269,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } - logInfo("All workers have launched.") + logInfo("All executors have launched.") // Launch a progress reporter thread, else the app will get killed after expiration // (def: 10mins) timeout. @@ -294,16 +294,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, val t = new Thread { override def run() { while (userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") + "max number of executor failures reached") } - val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - + val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning - yarnAllocator.getNumPendingAllocate - if (missingWorkerCount > 0) { + if (missingExecutorCount > 0) { logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingWorkerCount)) - yarnAllocator.addResourceRequests(missingWorkerCount) + format(missingExecutorCount)) + yarnAllocator.addResourceRequests(missingExecutorCount) } sendProgress() Thread.sleep(sleepTime) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala similarity index 92% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index f1c1fea0b5895..b697f103914fd 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -35,7 +35,7 @@ import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) +class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = @@ -93,7 +93,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar waitForSparkMaster() // Allocate all containers - allocateWorkers() + allocateExecutors() // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. @@ -175,7 +175,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar } - private def allocateWorkers() { + private def allocateExecutors() { // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = @@ -189,18 +189,18 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar preferredNodeLocationData, sparkConf) - logInfo("Allocating " + args.numWorkers + " workers.") + logInfo("Allocating " + args.numExecutors + " executors.") // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - yarnAllocator.addResourceRequests(args.numWorkers) - while ((yarnAllocator.getNumWorkersRunning < args.numWorkers) && (!driverClosed)) { + yarnAllocator.addResourceRequests(args.numExecutors) + while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { yarnAllocator.allocateResources() Thread.sleep(100) } - logInfo("All workers have launched.") + logInfo("All executors have launched.") } @@ -211,12 +211,12 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar val t = new Thread { override def run() { while (!driverClosed) { - val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - + val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning - yarnAllocator.getNumPendingAllocate - if (missingWorkerCount > 0) { + if (missingExecutorCount > 0) { logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingWorkerCount)) - yarnAllocator.addResourceRequests(missingWorkerCount) + format(missingExecutorCount)) + yarnAllocator.addResourceRequests(missingExecutorCount) } sendProgress() Thread.sleep(sleepTime) @@ -244,9 +244,9 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar } -object WorkerLauncher { +object ExecutorLauncher { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new WorkerLauncher(args).run() + new ExecutorLauncher(args).run() } } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala similarity index 90% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index ab4a79be70485..53c403f7d0913 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -38,16 +38,16 @@ import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} import org.apache.spark.{SparkConf, Logging} -class WorkerRunnable( +class ExecutorRunnable( container: Container, conf: Configuration, spConf: SparkConf, masterAddress: String, slaveId: String, hostname: String, - workerMemory: Int, - workerCores: Int) - extends Runnable with WorkerRunnableUtil with Logging { + executorMemory: Int, + executorCores: Int) + extends Runnable with ExecutorRunnableUtil with Logging { var rpc: YarnRPC = YarnRPC.create(conf) var nmClient: NMClient = _ @@ -55,7 +55,7 @@ class WorkerRunnable( val yarnConf: YarnConfiguration = new YarnConfiguration(conf) def run = { - logInfo("Starting Worker Container") + logInfo("Starting Executor Container") nmClient = NMClient.createNMClient() nmClient.init(yarnConf) nmClient.start() @@ -78,9 +78,9 @@ class WorkerRunnable( credentials.writeTokenStorageToStream(dob) ctx.setTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, workerMemory, workerCores) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) - logInfo("Setting up worker with commands: " + commands) + logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) // Send the start request to the ContainerManager diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 1ac61124cb028..e31c4060e8452 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -60,9 +60,9 @@ private[yarn] class YarnAllocationHandler( val conf: Configuration, val amClient: AMRMClient[ContainerRequest], val appAttemptId: ApplicationAttemptId, - val maxWorkers: Int, - val workerMemory: Int, - val workerCores: Int, + val maxExecutors: Int, + val executorMemory: Int, + val executorCores: Int, val preferredHostToCount: Map[String, Int], val preferredRackToCount: Map[String, Int], val sparkConf: SparkConf) @@ -89,20 +89,20 @@ private[yarn] class YarnAllocationHandler( // Number of container requests that have been sent to, but not yet allocated by the // ApplicationMaster. private val numPendingAllocate = new AtomicInteger() - private val numWorkersRunning = new AtomicInteger() - // Used to generate a unique id per worker - private val workerIdCounter = new AtomicInteger() + private val numExecutorsRunning = new AtomicInteger() + // Used to generate a unique id per executor + private val executorIdCounter = new AtomicInteger() private val lastResponseId = new AtomicInteger() - private val numWorkersFailed = new AtomicInteger() + private val numExecutorsFailed = new AtomicInteger() def getNumPendingAllocate: Int = numPendingAllocate.intValue - def getNumWorkersRunning: Int = numWorkersRunning.intValue + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue - def getNumWorkersFailed: Int = numWorkersFailed.intValue + def getNumExecutorsFailed: Int = numExecutorsFailed.intValue def isResourceConstraintSatisfied(container: Container): Boolean = { - container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + container.getResource.getMemory >= (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD) } def releaseContainer(container: Container) { @@ -127,13 +127,13 @@ private[yarn] class YarnAllocationHandler( logDebug(""" Allocated containers: %d - Current worker count: %d + Current executor count: %d Containers released: %s Containers to-be-released: %s Cluster resources: %s """.format( allocatedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers, allocateResponse.getAvailableResources)) @@ -240,64 +240,64 @@ private[yarn] class YarnAllocationHandler( // Run each of the allocated containers. for (container <- allocatedContainersToProcess) { - val numWorkersRunningNow = numWorkersRunning.incrementAndGet() - val workerHostname = container.getNodeId.getHost + val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() + val executorHostname = container.getNodeId.getHost val containerId = container.getId - val workerMemoryOverhead = (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) - assert(container.getResource.getMemory >= workerMemoryOverhead) + val executorMemoryOverhead = (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + assert(container.getResource.getMemory >= executorMemoryOverhead) - if (numWorkersRunningNow > maxWorkers) { + if (numExecutorsRunningNow > maxExecutors) { logInfo("""Ignoring container %s at host %s, since we already have the required number of - containers for it.""".format(containerId, workerHostname)) + containers for it.""".format(containerId, executorHostname)) releaseContainer(container) - numWorkersRunning.decrementAndGet() + numExecutorsRunning.decrementAndGet() } else { - val workerId = workerIdCounter.incrementAndGet().toString + val executorId = executorIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - logInfo("Launching container %s for on host %s".format(containerId, workerHostname)) + logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) // To be safe, remove the container from `pendingReleaseContainers`. pendingReleaseContainers.remove(containerId) - val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) + val rack = YarnAllocationHandler.lookupRack(conf, executorHostname) allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]()) containerSet += containerId - allocatedContainerToHostMap.put(containerId, workerHostname) + allocatedContainerToHostMap.put(containerId, executorHostname) if (rack != null) { allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) } } - logInfo("Launching WorkerRunnable. driverUrl: %s, workerHostname: %s".format(driverUrl, workerHostname)) - val workerRunnable = new WorkerRunnable( + logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format(driverUrl, executorHostname)) + val executorRunnable = new ExecutorRunnable( container, conf, sparkConf, driverUrl, - workerId, - workerHostname, - workerMemory, - workerCores) - new Thread(workerRunnable).start() + executorId, + executorHostname, + executorMemory, + executorCores) + new Thread(executorRunnable).start() } } logDebug(""" Finished allocating %s containers (from %s originally). - Current number of workers running: %d, + Current number of executors running: %d, releasedContainerList: %s, pendingReleaseContainers: %s """.format( allocatedContainersToProcess, allocatedContainers, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers)) } @@ -314,9 +314,9 @@ private[yarn] class YarnAllocationHandler( // `pendingReleaseContainers`. pendingReleaseContainers.remove(containerId) } else { - // Decrement the number of workers running. The next iteration of the ApplicationMaster's + // Decrement the number of executors running. The next iteration of the ApplicationMaster's // reporting thread will take care of allocating. - numWorkersRunning.decrementAndGet() + numExecutorsRunning.decrementAndGet() logInfo("Completed container %s (state: %s, exit status: %s)".format( containerId, completedContainer.getState, @@ -326,7 +326,7 @@ private[yarn] class YarnAllocationHandler( // now I think its ok as none of the containers are expected to exit if (completedContainer.getExitStatus() != 0) { logInfo("Container marked as failed: " + containerId) - numWorkersFailed.incrementAndGet() + numExecutorsFailed.incrementAndGet() } } @@ -364,12 +364,12 @@ private[yarn] class YarnAllocationHandler( } logDebug(""" Finished processing %d completed containers. - Current number of workers running: %d, + Current number of executors running: %d, releasedContainerList: %s, pendingReleaseContainers: %s """.format( completedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers)) } @@ -421,18 +421,18 @@ private[yarn] class YarnAllocationHandler( retval } - def addResourceRequests(numWorkers: Int) { + def addResourceRequests(numExecutors: Int) { val containerRequests: List[ContainerRequest] = - if (numWorkers <= 0 || preferredHostToCount.isEmpty) { - logDebug("numWorkers: " + numWorkers + ", host preferences: " + + if (numExecutors <= 0 || preferredHostToCount.isEmpty) { + logDebug("numExecutors: " + numExecutors + ", host preferences: " + preferredHostToCount.isEmpty) createResourceRequests( AllocationType.ANY, resource = null, - numWorkers, + numExecutors, YarnAllocationHandler.PRIORITY).toList } else { - // Request for all hosts in preferred nodes and for numWorkers - + // Request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) for ((candidateHost, candidateCount) <- preferredHostToCount) { @@ -452,7 +452,7 @@ private[yarn] class YarnAllocationHandler( val anyContainerRequests = createResourceRequests( AllocationType.ANY, resource = null, - numWorkers, + numExecutors, YarnAllocationHandler.PRIORITY) val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( @@ -468,11 +468,11 @@ private[yarn] class YarnAllocationHandler( amClient.addContainerRequest(request) } - if (numWorkers > 0) { - numPendingAllocate.addAndGet(numWorkers) - logInfo("Will Allocate %d worker containers, each with %d memory".format( - numWorkers, - (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))) + if (numExecutors > 0) { + numPendingAllocate.addAndGet(numExecutors) + logInfo("Will Allocate %d executor containers, each with %d memory".format( + numExecutors, + (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD))) } else { logDebug("Empty allocation request ...") } @@ -494,7 +494,7 @@ private[yarn] class YarnAllocationHandler( private def createResourceRequests( requestType: AllocationType.AllocationType, resource: String, - numWorkers: Int, + numExecutors: Int, priority: Int ): ArrayBuffer[ContainerRequest] = { @@ -507,7 +507,7 @@ private[yarn] class YarnAllocationHandler( val nodeLocal = constructContainerRequests( Array(hostname), racks = null, - numWorkers, + numExecutors, priority) // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. @@ -516,10 +516,10 @@ private[yarn] class YarnAllocationHandler( } case AllocationType.RACK => { val rack = resource - constructContainerRequests(hosts = null, Array(rack), numWorkers, priority) + constructContainerRequests(hosts = null, Array(rack), numExecutors, priority) } case AllocationType.ANY => constructContainerRequests( - hosts = null, racks = null, numWorkers, priority) + hosts = null, racks = null, numExecutors, priority) case _ => throw new IllegalArgumentException( "Unexpected/unsupported request type: " + requestType) } @@ -528,18 +528,18 @@ private[yarn] class YarnAllocationHandler( private def constructContainerRequests( hosts: Array[String], racks: Array[String], - numWorkers: Int, + numExecutors: Int, priority: Int ): ArrayBuffer[ContainerRequest] = { - val memoryRequest = workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD - val resource = Resource.newInstance(memoryRequest, workerCores) + val memoryRequest = executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD + val resource = Resource.newInstance(memoryRequest, executorCores) val prioritySetting = Records.newRecord(classOf[Priority]) prioritySetting.setPriority(priority) val requests = new ArrayBuffer[ContainerRequest]() - for (i <- 0 until numWorkers) { + for (i <- 0 until numExecutors) { requests += new ContainerRequest(resource, hosts, racks, prioritySetting) } requests @@ -574,9 +574,9 @@ object YarnAllocationHandler { conf, amClient, appAttemptId, - args.numWorkers, - args.workerMemory, - args.workerCores, + args.numExecutors, + args.executorMemory, + args.executorCores, Map[String, Int](), Map[String, Int](), sparkConf) @@ -596,9 +596,9 @@ object YarnAllocationHandler { conf, amClient, appAttemptId, - args.numWorkers, - args.workerMemory, - args.workerCores, + args.numExecutors, + args.executorMemory, + args.executorCores, hostToSplitCount, rackToSplitCount, sparkConf) @@ -608,9 +608,9 @@ object YarnAllocationHandler { conf: Configuration, amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, - maxWorkers: Int, - workerMemory: Int, - workerCores: Int, + maxExecutors: Int, + executorMemory: Int, + executorCores: Int, map: collection.Map[String, collection.Set[SplitInfo]], sparkConf: SparkConf ): YarnAllocationHandler = { @@ -619,9 +619,9 @@ object YarnAllocationHandler { conf, amClient, appAttemptId, - maxWorkers, - workerMemory, - workerCores, + maxExecutors, + executorMemory, + executorCores, hostToCount, rackToCount, sparkConf) From ca4bf8c572c2f70b484830f1db414b5073744ab6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 13 Mar 2014 12:16:04 -0700 Subject: [PATCH 14/23] SPARK-1236 - Upgrade Jetty to 9.1.3.v20140225. Author: Reynold Xin Closes #113 from rxin/jetty9 and squashes the following commits: 867a2ce [Reynold Xin] Updated Jetty version to 9.1.3.v20140225 in Maven build file. d7c97ca [Reynold Xin] Return the correctly bound port. d14706f [Reynold Xin] Upgrade Jetty to 9.1.3.v20140225. --- .../scala/org/apache/spark/HttpServer.scala | 21 ++++--- .../org/apache/spark/ui/JettyUtils.scala | 58 ++++++++++--------- pom.xml | 8 +-- project/SparkBuild.scala | 8 +-- 4 files changed, 49 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index cb5df25fa48df..d14693cb78a2d 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -21,10 +21,9 @@ import java.io.File import org.eclipse.jetty.util.security.{Constraint, Password} import org.eclipse.jetty.security.authentication.DigestAuthenticator -import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService, SecurityHandler} +import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService} -import org.eclipse.jetty.server.Server -import org.eclipse.jetty.server.bio.SocketConnector +import org.eclipse.jetty.server.{Server, ServerConnector} import org.eclipse.jetty.server.handler.{DefaultHandler, HandlerList, ResourceHandler} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -43,7 +42,7 @@ private[spark] class ServerStateException(message: String) extends Exception(mes */ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityManager) extends Logging { - private var server: Server = null + private var server: Server = _ private var port: Int = -1 def start() { @@ -51,16 +50,16 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan throw new ServerStateException("Server is already started") } else { logInfo("Starting HTTP Server") - server = new Server() - val connector = new SocketConnector - connector.setMaxIdleTime(60*1000) + val threadPool = new QueuedThreadPool + threadPool.setDaemon(true) + + server = new Server(threadPool) + val connector = new ServerConnector(server) + connector.setIdleTimeout(60 * 1000) connector.setSoLingerTime(-1) connector.setPort(0) server.addConnector(connector) - val threadPool = new QueuedThreadPool - threadPool.setDaemon(true) - server.setThreadPool(threadPool) val resHandler = new ResourceHandler resHandler.setResourceBase(resourceBase.getAbsolutePath) @@ -79,7 +78,7 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan } server.start() - port = server.getConnectors()(0).getLocalPort() + port = connector.getLocalPort } } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 7c35cd165ad7c..cc7324939668b 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.ui import java.net.InetSocketAddress import java.net.URL +import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletResponse, HttpServletRequest} import scala.annotation.tailrec @@ -28,7 +29,7 @@ import scala.xml.Node import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} -import org.eclipse.jetty.server.{DispatcherType, Server} +import org.eclipse.jetty.server.{NetworkConnector, Server} import org.eclipse.jetty.server.handler.HandlerList import org.eclipse.jetty.servlet.{DefaultServlet, FilterHolder, ServletContextHandler, ServletHolder} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -60,8 +61,7 @@ private[spark] object JettyUtils extends Logging { def createServlet[T <% AnyRef](servletParams: ServletParams[T], securityMgr: SecurityManager): HttpServlet = { new HttpServlet { - override def doGet(request: HttpServletRequest, - response: HttpServletResponse) { + override def doGet(request: HttpServletRequest, response: HttpServletResponse) { if (securityMgr.checkUIViewPermissions(request.getRemoteUser())) { response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) response.setStatus(HttpServletResponse.SC_OK) @@ -72,7 +72,7 @@ private[spark] object JettyUtils extends Logging { response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.sendError(HttpServletResponse.SC_UNAUTHORIZED, - "User is not authorized to access this page."); + "User is not authorized to access this page.") } } } @@ -120,26 +120,25 @@ private[spark] object JettyUtils extends Logging { private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) - filters.foreach { - case filter : String => - if (!filter.isEmpty) { - logInfo("Adding filter: " + filter) - val holder : FilterHolder = new FilterHolder() - holder.setClassName(filter) - // get any parameters for each filter - val paramName = "spark." + filter + ".params" - val params = conf.get(paramName, "").split(',').map(_.trim()).toSet - params.foreach { - case param : String => - if (!param.isEmpty) { - val parts = param.split("=") - if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) - } - } - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, - DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) - handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } + filters.foreach { filter => + if (!filter.isEmpty) { + logInfo("Adding filter: " + filter) + val holder : FilterHolder = new FilterHolder() + holder.setClassName(filter) + // get any parameters for each filter + val paramName = "spark." + filter + ".params" + val params = conf.get(paramName, "").split(',').map(_.trim()).toSet + params.foreach { + case param : String => + if (!param.isEmpty) { + val parts = param.split("=") + if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) + } } + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, + DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) + handlers.foreach { handler => handler.addFilter(holder, "/*", enumDispatcher) } + } } } @@ -150,7 +149,10 @@ private[spark] object JettyUtils extends Logging { * If the desired port number is contented, continues incrementing ports until a free port is * found. Returns the chosen port and the jetty Server object. */ - def startJettyServer(hostName: String, port: Int, handlers: Seq[ServletContextHandler], + def startJettyServer( + hostName: String, + port: Int, + handlers: Seq[ServletContextHandler], conf: SparkConf): (Server, Int) = { addFilters(handlers, conf) @@ -160,16 +162,18 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { val server = new Server(new InetSocketAddress(hostName, currentPort)) - val pool = new QueuedThreadPool + // Unfortunately Jetty 9 doesn't allow us to set both the thread pool and the port number in + // constructor. But fortunately the pool allocated by Jetty is always a QueuedThreadPool. + val pool = server.getThreadPool.asInstanceOf[QueuedThreadPool] pool.setDaemon(true) - server.setThreadPool(pool) + server.setHandler(handlerList) Try { server.start() } match { case s: Success[_] => - (server, server.getConnectors.head.getLocalPort) + (server, server.getConnectors.head.asInstanceOf[NetworkConnector].getLocalPort) case f: Failure[_] => server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) diff --git a/pom.xml b/pom.xml index d2c36dc260fa4..441e931cc8e77 100644 --- a/pom.xml +++ b/pom.xml @@ -158,22 +158,22 @@ org.eclipse.jetty jetty-util - 7.6.8.v20121106 + 9.1.3.v20140225 org.eclipse.jetty jetty-security - 7.6.8.v20121106 + 9.1.3.v20140225 org.eclipse.jetty jetty-plus - 7.6.8.v20121106 + 9.1.3.v20140225 org.eclipse.jetty jetty-server - 7.6.8.v20121106 + 9.1.3.v20140225 com.google.guava diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b0c3bf29dfd4f..bfc16e814b845 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -232,10 +232,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", - "org.eclipse.jetty" % "jetty-util" % "7.6.8.v20121106", - "org.eclipse.jetty" % "jetty-plus" % "7.6.8.v20121106", - "org.eclipse.jetty" % "jetty-security" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-server" % "9.1.3.v20140225", + "org.eclipse.jetty" % "jetty-util" % "9.1.3.v20140225", + "org.eclipse.jetty" % "jetty-plus" % "9.1.3.v20140225", + "org.eclipse.jetty" % "jetty-security" % "9.1.3.v20140225", /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"), "org.scalatest" %% "scalatest" % "1.9.1" % "test", From 181b130a0c2a0752009fdf5602e5d6d87f5b1212 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Thu, 13 Mar 2014 20:27:36 -0700 Subject: [PATCH 15/23] [bugfix] wrong client arg, should use executor-cores client arg is wrong, it should be executor-cores. it causes executor fail to start when executor-cores is specified Author: Tianshuo Deng Closes #138 from tsdeng/bugfix_wrong_client_args and squashes the following commits: 304826d [Tianshuo Deng] wrong client arg, should use executor-cores --- .../scala/org/apache/spark/deploy/yarn/ClientArguments.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 a001060cdb746..c565f2dde24fc 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 @@ -95,7 +95,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { executorMemory = value args = tail - case ("--worker-cores" | "--executor-memory") :: IntParam(value) :: tail => + case ("--worker-cores" | "--executor-cores") :: IntParam(value) :: tail => if (args(0) == "--worker-cores") { println("--worker-cores is deprecated. Use --executor-cores instead.") } From e19044cb1048c3755d1ea2cb43879d2225d49b54 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 14 Mar 2014 11:40:26 -0700 Subject: [PATCH 16/23] Fix serialization of MutablePair. Also provide an interface for easy updating. Author: Michael Armbrust Closes #141 from marmbrus/mutablePair and squashes the following commits: f5c4783 [Michael Armbrust] Change function name to update 8bfd973 [Michael Armbrust] Fix serialization of MutablePair. Also provide an interface for easy updating. --- .../scala/org/apache/spark/util/MutablePair.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala index b053266f12748..2c1a6f8fd0a44 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -25,10 +25,20 @@ package org.apache.spark.util * @param _2 Element 2 of this MutablePair */ case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T1, - @specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T2] + @specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T2] (var _1: T1, var _2: T2) extends Product2[T1, T2] { + /** No-arg constructor for serialization */ + def this() = this(null.asInstanceOf[T1], null.asInstanceOf[T2]) + + /** Updates this pair with new values and returns itself */ + def update(n1: T1, n2: T2): MutablePair[T1, T2] = { + _1 = n1 + _2 = n2 + this + } + override def toString = "(" + _1 + "," + _2 + ")" override def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[_,_]] From 97e4459e1e4cca8696535e10a91733c15f960107 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 15 Mar 2014 16:43:27 -0700 Subject: [PATCH 17/23] SPARK-1254. Consolidate, order, and harmonize repository declarations in Maven/SBT builds This suggestion addresses a few minor suboptimalities with how repositories are handled. 1) Use HTTPS consistently to access repos, instead of HTTP 2) Consolidate repository declarations in the parent POM file, in the case of the Maven build, so that their ordering can be controlled to put the fully optional Cloudera repo at the end, after required repos. (This was prompted by the untimely failure of the Cloudera repo this week, which made the Spark build fail. #2 would have prevented that.) 3) Update SBT build to match Maven build in this regard 4) Update SBT build to not refer to Sonatype snapshot repos. This wasn't in Maven, and a build generally would not refer to external snapshots, but I'm not 100% sure on this one. Author: Sean Owen Closes #145 from srowen/SPARK-1254 and squashes the following commits: 42f9bfc [Sean Owen] Use HTTPS for repos; consolidate repos in parent in order to put optional Cloudera repo last; harmonize SBT build repos with Maven; remove snapshot repos from SBT build which weren't in Maven --- examples/pom.xml | 15 --------------- external/mqtt/pom.xml | 14 -------------- pom.xml | 32 ++++++++++++++++++++++++++++++-- project/SparkBuild.scala | 23 ++++++++++++----------- streaming/pom.xml | 14 -------------- 5 files changed, 42 insertions(+), 56 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 9f0e2d0b875b8..382a38d9400b9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -45,21 +45,6 @@ - - - apache-repo - Apache Repository - https://repository.apache.org/content/repositories/releases - - true - - - false - - - - - org.apache.spark diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 06c751df7f9c8..3710a63541d78 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -45,20 +45,6 @@ - - - mqtt-repo - MQTT Repository - https://repo.eclipse.org/content/repositories/paho-releases - - true - - - false - - - - org.apache.spark diff --git a/pom.xml b/pom.xml index 441e931cc8e77..377b6463136dd 100644 --- a/pom.xml +++ b/pom.xml @@ -127,7 +127,18 @@ maven-repo Maven Repository - http://repo.maven.apache.org/maven2 + https://repo.maven.apache.org/maven2 + + true + + + false + + + + apache-repo + Apache Repository + https://repository.apache.org/content/repositories/releases true @@ -138,7 +149,18 @@ jboss-repo JBoss Repository - http://repository.jboss.org/nexus/content/repositories/releases + https://repository.jboss.org/nexus/content/repositories/releases + + true + + + false + + + + mqtt-repo + MQTT Repository + https://repo.eclipse.org/content/repositories/paho-releases true @@ -150,6 +172,12 @@ cloudera-repo Cloudera Repository https://repository.cloudera.com/artifactory/cloudera-repos + + true + + + false + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index bfc16e814b845..db24f124f3cf8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -174,12 +174,18 @@ object SparkBuild extends Build { // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), - // also check the local Maven repository ~/.m2 - resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))), - - // For Sonatype publishing - resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", - "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"), + resolvers ++= Seq( + "Maven Repository" at "https://repo.maven.apache.org/maven2", + "Apache Repository" at "https://repository.apache.org/content/repositories/releases", + "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", + "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", + "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", + // For Sonatype publishing + //"sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", + //"sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", + // also check the local Maven repository ~/.m2 + Resolver.mavenLocal + ), publishMavenStyle := true, @@ -272,10 +278,6 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", - resolvers ++= Seq( - "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" - ), libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", @@ -470,7 +472,6 @@ object SparkBuild extends Build { def mqttSettings() = streamingSettings ++ Seq( name := "spark-streaming-mqtt", - resolvers ++= Seq("Eclipse Repo" at "https://repo.eclipse.org/content/repositories/paho-releases/"), libraryDependencies ++= Seq("org.eclipse.paho" % "mqtt-client" % "0.4.0") ) } diff --git a/streaming/pom.xml b/streaming/pom.xml index 2343e381e6f7c..2cb8bde6642bc 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -45,20 +45,6 @@ - - - apache-repo - Apache Repository - https://repository.apache.org/content/repositories/releases - - true - - - false - - - - org.apache.spark From f5486e9f75d62919583da5ecf9a9ad00222b2227 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 16 Mar 2014 09:57:21 -0700 Subject: [PATCH 18/23] SPARK-1255: Allow user to pass Serializer object instead of class name for shuffle. This is more general than simply passing a string name and leaves more room for performance optimizations. Note that this is technically an API breaking change in the following two ways: 1. The shuffle serializer specification in ShuffleDependency now require an object instead of a String (of the class name), but I suspect nobody else in this world has used this API other than me in GraphX and Shark. 2. Serializer's in Spark from now on are required to be serializable. Author: Reynold Xin Closes #149 from rxin/serializer and squashes the following commits: 5acaccd [Reynold Xin] Properly call serializer's constructors. 2a8d75a [Reynold Xin] Added more documentation for the serializer option in ShuffleDependency. 7420185 [Reynold Xin] Allow user to pass Serializer object instead of class name for shuffle. --- .../scala/org/apache/spark/Dependency.scala | 6 +- .../org/apache/spark/ShuffleFetcher.scala | 2 +- .../scala/org/apache/spark/SparkEnv.scala | 24 +++--- .../org/apache/spark/rdd/CoGroupedRDD.scala | 18 ++--- .../apache/spark/rdd/PairRDDFunctions.scala | 7 +- .../org/apache/spark/rdd/ShuffledRDD.scala | 13 ++-- .../org/apache/spark/rdd/SubtractedRDD.scala | 20 ++--- .../spark/scheduler/ShuffleMapTask.scala | 3 +- .../spark/serializer/JavaSerializer.scala | 27 ++++--- .../spark/serializer/KryoSerializer.scala | 16 ++-- .../apache/spark/serializer/Serializer.scala | 16 +++- .../spark/serializer/SerializerManager.scala | 75 ------------------- .../collection/ExternalAppendOnlyMap.scala | 2 +- .../scala/org/apache/spark/ShuffleSuite.scala | 9 ++- .../apache/spark/graphx/impl/GraphImpl.scala | 2 +- .../graphx/impl/MessageToPartition.scala | 12 +-- .../spark/graphx/impl/Serializers.scala | 14 ++-- .../apache/spark/graphx/SerializerSuite.scala | 30 +++----- 18 files changed, 125 insertions(+), 171 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index cc30105940d1a..448f87b81ef4a 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -18,6 +18,7 @@ package org.apache.spark import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer /** * Base class for dependencies. @@ -43,12 +44,13 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output - * @param serializerClass class name of the serializer to use + * @param serializer [[Serializer]] to use. If set to null, the default serializer, as specified + * by `spark.serializer` config option, will be used. */ class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, - val serializerClass: String = null) + val serializer: Serializer = null) extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() diff --git a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala index e8f756c408889..a4f69b6b22b2c 100644 --- a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala @@ -29,7 +29,7 @@ private[spark] abstract class ShuffleFetcher { shuffleId: Int, reduceId: Int, context: TaskContext, - serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[T] + serializer: Serializer = SparkEnv.get.serializer): Iterator[T] /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5e43b5198422c..d035d909b7b25 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -28,7 +28,7 @@ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.storage.{BlockManager, BlockManagerMaster, BlockManagerMasterActor} import org.apache.spark.network.ConnectionManager -import org.apache.spark.serializer.{Serializer, SerializerManager} +import org.apache.spark.serializer.Serializer import org.apache.spark.util.{AkkaUtils, Utils} /** @@ -41,7 +41,6 @@ import org.apache.spark.util.{AkkaUtils, Utils} class SparkEnv private[spark] ( val executorId: String, val actorSystem: ActorSystem, - val serializerManager: SerializerManager, val serializer: Serializer, val closureSerializer: Serializer, val cacheManager: CacheManager, @@ -139,16 +138,22 @@ object SparkEnv extends Logging { // defaultClassName if the property is not set, and return it as a T def instantiateClass[T](propertyName: String, defaultClassName: String): T = { val name = conf.get(propertyName, defaultClassName) - Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] + val cls = Class.forName(name, true, classLoader) + // First try with the constructor that takes SparkConf. If we can't find one, + // use a no-arg constructor instead. + try { + cls.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[T] + } catch { + case _: NoSuchMethodException => + cls.getConstructor().newInstance().asInstanceOf[T] + } } - val serializerManager = new SerializerManager - val serializer = serializerManager.setDefault( - conf.get("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) + val serializer = instantiateClass[Serializer]( + "spark.serializer", "org.apache.spark.serializer.JavaSerializer") - val closureSerializer = serializerManager.get( - conf.get("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), - conf) + val closureSerializer = instantiateClass[Serializer]( + "spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer") def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { @@ -220,7 +225,6 @@ object SparkEnv extends Logging { new SparkEnv( executorId, actorSystem, - serializerManager, serializer, closureSerializer, cacheManager, diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 699a10c96c227..8561711931047 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -24,6 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} +import org.apache.spark.serializer.Serializer private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -66,10 +67,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Seq[CoGroup] - private var serializerClass: String = null + private var serializer: Serializer = null - def setSerializer(cls: String): CoGroupedRDD[K] = { - serializerClass = cls + def setSerializer(serializer: Serializer): CoGroupedRDD[K] = { + this.serializer = serializer this } @@ -80,7 +81,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: new OneToOneDependency(rdd) } else { logDebug("Adding shuffle dependency with " + rdd) - new ShuffleDependency[Any, Any](rdd, part, serializerClass) + new ShuffleDependency[Any, Any](rdd, part, serializer) } } } @@ -113,18 +114,17 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: // A list of (rdd iterator, dependency number) pairs val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)] for ((dep, depNum) <- split.deps.zipWithIndex) dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => // Read them from the parent val it = rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]] rddIterators += ((it, depNum)) - } - case ShuffleCoGroupSplitDep(shuffleId) => { + + case ShuffleCoGroupSplitDep(shuffleId) => // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher - val ser = SparkEnv.get.serializerManager.get(serializerClass, sparkConf) + val ser = Serializer.getSerializer(serializer) val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) rddIterators += ((it, depNum)) - } } if (!externalSorting) { 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 b20ed99f89f34..b0d322fe27bd5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -44,6 +44,7 @@ import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.serializer.Serializer import org.apache.spark.util.SerializableHyperLogLog /** @@ -73,7 +74,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) mergeCombiners: (C, C) => C, partitioner: Partitioner, mapSideCombine: Boolean = true, - serializerClass: String = null): RDD[(K, C)] = { + serializer: Serializer = null): RDD[(K, C)] = { require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0 if (getKeyClass().isArray) { if (mapSideCombine) { @@ -93,13 +94,13 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) aggregator.combineValuesByKey(iter, context) }, preservesPartitioning = true) val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner) - .setSerializer(serializerClass) + .setSerializer(serializer) partitioned.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineCombinersByKey(iter, context)) }, preservesPartitioning = true) } else { // Don't apply map-side combiner. - val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass) + val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializer) values.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) }, preservesPartitioning = true) diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 0bbda25a905cd..02660ea6a45c5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { override val index = idx @@ -38,15 +39,15 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( part: Partitioner) extends RDD[P](prev.context, Nil) { - private var serializerClass: String = null + private var serializer: Serializer = null - def setSerializer(cls: String): ShuffledRDD[K, V, P] = { - serializerClass = cls + def setSerializer(serializer: Serializer): ShuffledRDD[K, V, P] = { + this.serializer = serializer this } override def getDependencies: Seq[Dependency[_]] = { - List(new ShuffleDependency(prev, part, serializerClass)) + List(new ShuffleDependency(prev, part, serializer)) } override val partitioner = Some(part) @@ -57,8 +58,8 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[P] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context, - SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)) + val ser = Serializer.getSerializer(serializer) + SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context, ser) } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index 5fe9f363db453..9a09c05bbc959 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -30,6 +30,7 @@ import org.apache.spark.Partitioner import org.apache.spark.ShuffleDependency import org.apache.spark.SparkEnv import org.apache.spark.TaskContext +import org.apache.spark.serializer.Serializer /** * An optimized version of cogroup for set difference/subtraction. @@ -53,10 +54,10 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { - private var serializerClass: String = null + private var serializer: Serializer = null - def setSerializer(cls: String): SubtractedRDD[K, V, W] = { - serializerClass = cls + def setSerializer(serializer: Serializer): SubtractedRDD[K, V, W] = { + this.serializer = serializer this } @@ -67,7 +68,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( new OneToOneDependency(rdd) } else { logDebug("Adding shuffle dependency with " + rdd) - new ShuffleDependency(rdd, part, serializerClass) + new ShuffleDependency(rdd, part, serializer) } } } @@ -92,7 +93,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] - val serializer = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf) + val ser = Serializer.getSerializer(serializer) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) @@ -105,14 +106,13 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( } } def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit) = dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op) - } - case ShuffleCoGroupSplitDep(shuffleId) => { + + case ShuffleCoGroupSplitDep(shuffleId) => val iter = SparkEnv.get.shuffleFetcher.fetch[Product2[K, V]](shuffleId, partition.index, - context, serializer) + context, ser) iter.foreach(op) - } } // the first dep is rdd1; add all values to the map integrate(partition.deps(0), t => getSeq(t._1) += t._2) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 77789031f464a..2a9edf4a76b97 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -26,6 +26,7 @@ import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDDCheckpointData +import org.apache.spark.serializer.Serializer import org.apache.spark.storage._ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} @@ -153,7 +154,7 @@ private[spark] class ShuffleMapTask( try { // Obtain all the block writers for shuffle blocks. - val ser = SparkEnv.get.serializerManager.get(dep.serializerClass, SparkEnv.get.conf) + val ser = Serializer.getSerializer(dep.serializer) shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser) // Write the map output to its associated buckets. 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 bfa647f7f0516..18a68b05fa853 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -23,11 +23,10 @@ import java.nio.ByteBuffer import org.apache.spark.SparkConf import org.apache.spark.util.ByteBufferInputStream -private[spark] class JavaSerializationStream(out: OutputStream, conf: SparkConf) +private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) extends SerializationStream { - val objOut = new ObjectOutputStream(out) - var counter = 0 - val counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) + private val objOut = new ObjectOutputStream(out) + private var counter = 0 /** * Calling reset to avoid memory leak: @@ -51,7 +50,7 @@ private[spark] class JavaSerializationStream(out: OutputStream, conf: SparkConf) private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoader) extends DeserializationStream { - val objIn = new ObjectInputStream(in) { + private val objIn = new ObjectInputStream(in) { override def resolveClass(desc: ObjectStreamClass) = Class.forName(desc.getName, false, loader) } @@ -60,7 +59,7 @@ extends DeserializationStream { def close() { objIn.close() } } -private[spark] class JavaSerializerInstance(conf: SparkConf) extends SerializerInstance { +private[spark] class JavaSerializerInstance(counterReset: Int) extends SerializerInstance { def serialize[T](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) @@ -82,7 +81,7 @@ private[spark] class JavaSerializerInstance(conf: SparkConf) extends SerializerI } def serializeStream(s: OutputStream): SerializationStream = { - new JavaSerializationStream(s, conf) + new JavaSerializationStream(s, counterReset) } def deserializeStream(s: InputStream): DeserializationStream = { @@ -97,6 +96,16 @@ private[spark] class JavaSerializerInstance(conf: SparkConf) extends SerializerI /** * A Spark serializer that uses Java's built-in serialization. */ -class JavaSerializer(conf: SparkConf) extends Serializer { - def newInstance(): SerializerInstance = new JavaSerializerInstance(conf) +class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { + private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) + + def newInstance(): SerializerInstance = new JavaSerializerInstance(counterReset) + + override def writeExternal(out: ObjectOutput) { + out.writeInt(counterReset) + } + + override def readExternal(in: ObjectInput) { + counterReset = in.readInt() + } } 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 920490f9d0d61..6b6d814c1fe92 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -34,10 +34,14 @@ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. */ -class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging { - private val bufferSize = { - conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024 - } +class KryoSerializer(conf: SparkConf) + extends org.apache.spark.serializer.Serializer + with Logging + with Serializable { + + private val bufferSize = conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024 + private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) + private val registrator = conf.getOption("spark.kryo.registrator") def newKryoOutput() = new KryoOutput(bufferSize) @@ -48,7 +52,7 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(conf.getBoolean("spark.kryo.referenceTracking", true)) + kryo.setReferences(referenceTracking) for (cls <- KryoSerializer.toRegister) kryo.register(cls) @@ -58,7 +62,7 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial // Allow the user to register their own classes by setting spark.kryo.registrator try { - for (regCls <- conf.getOption("spark.kryo.registrator")) { + for (regCls <- registrator) { logDebug("Running user registrator: " + regCls) val reg = Class.forName(regCls, true, classLoader).newInstance() .asInstanceOf[KryoRegistrator] 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 16677ab54be04..099143494b851 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -23,21 +23,31 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import org.apache.spark.util.{ByteBufferInputStream, NextIterator} +import org.apache.spark.SparkEnv /** * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual * serialization and are guaranteed to only be called from one thread at a time. * - * Implementations of this trait should have a zero-arg constructor or a constructor that accepts a - * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes - * precedence. + * Implementations of this trait should implement: + * 1. a zero-arg constructor or a constructor that accepts a [[org.apache.spark.SparkConf]] + * as parameter. If both constructors are defined, the latter takes precedence. + * + * 2. Java serialization interface. */ trait Serializer { def newInstance(): SerializerInstance } +object Serializer { + def getSerializer(serializer: Serializer): Serializer = { + if (serializer == null) SparkEnv.get.serializer else serializer + } +} + + /** * An instance of a serializer, for use by one thread at a time. */ diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala deleted file mode 100644 index 65ac0155f45e7..0000000000000 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ /dev/null @@ -1,75 +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.serializer - -import java.util.concurrent.ConcurrentHashMap - -import org.apache.spark.SparkConf - -/** - * A service that returns a serializer object given the serializer's class name. If a previous - * instance of the serializer object has been created, the get method returns that instead of - * creating a new one. - */ -private[spark] class SerializerManager { - // TODO: Consider moving this into SparkConf itself to remove the global singleton. - - private val serializers = new ConcurrentHashMap[String, Serializer] - private var _default: Serializer = _ - - def default = _default - - def setDefault(clsName: String, conf: SparkConf): Serializer = { - _default = get(clsName, conf) - _default - } - - def get(clsName: String, conf: SparkConf): Serializer = { - if (clsName == null) { - default - } else { - var serializer = serializers.get(clsName) - if (serializer != null) { - // If the serializer has been created previously, reuse that. - serializer - } else this.synchronized { - // Otherwise, create a new one. But make sure no other thread has attempted - // to create another new one at the same time. - serializer = serializers.get(clsName) - if (serializer == null) { - val clsLoader = Thread.currentThread.getContextClassLoader - val cls = Class.forName(clsName, true, clsLoader) - - // First try with the constructor that takes SparkConf. If we can't find one, - // use a no-arg constructor instead. - try { - val constructor = cls.getConstructor(classOf[SparkConf]) - serializer = constructor.newInstance(conf).asInstanceOf[Serializer] - } catch { - case _: NoSuchMethodException => - val constructor = cls.getConstructor() - serializer = constructor.newInstance().asInstanceOf[Serializer] - } - - serializers.put(clsName, serializer) - } - serializer - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index ed74a31f05bae..caa06d5b445b4 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -60,7 +60,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, - serializer: Serializer = SparkEnv.get.serializerManager.default, + serializer: Serializer = SparkEnv.get.serializer, blockManager: BlockManager = SparkEnv.get.blockManager) extends Iterable[(K, C)] with Serializable with Logging { diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index abea36f7c83df..be6508a40ea61 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -27,6 +27,9 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { + + val conf = new SparkConf(loadDefaults = false) + test("groupByKey without compression") { try { System.setProperty("spark.shuffle.compress", "false") @@ -54,7 +57,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // If the Kryo serializer is not used correctly, the shuffle would fail because the // default Java serializer cannot handle the non serializable class. val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( - b, new HashPartitioner(NUM_BLOCKS)).setSerializer(classOf[KryoSerializer].getName) + b, new HashPartitioner(NUM_BLOCKS)).setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 10) @@ -76,7 +79,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // If the Kryo serializer is not used correctly, the shuffle would fail because the // default Java serializer cannot handle the non serializable class. val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( - b, new HashPartitioner(3)).setSerializer(classOf[KryoSerializer].getName) + b, new HashPartitioner(3)).setSerializer(new KryoSerializer(conf)) assert(c.count === 10) } @@ -92,7 +95,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10)) - .setSerializer(classOf[KryoSerializer].getName) + .setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 4) 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 1d029bf009e8c..5e9be18990ba3 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 @@ -391,6 +391,6 @@ object GraphImpl { // TODO: Consider doing map side distinct before shuffle. new ShuffledRDD[VertexId, Int, (VertexId, Int)]( edges.collectVertexIds.map(vid => (vid, 0)), partitioner) - .setSerializer(classOf[VertexIdMsgSerializer].getName) + .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 e9ee09c3614c1..fe6fe76defdc5 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 @@ -65,11 +65,11 @@ class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { - rdd.setSerializer(classOf[IntVertexBroadcastMsgSerializer].getName) + rdd.setSerializer(new IntVertexBroadcastMsgSerializer) } else if (classTag[T] == ClassTag.Long) { - rdd.setSerializer(classOf[LongVertexBroadcastMsgSerializer].getName) + rdd.setSerializer(new LongVertexBroadcastMsgSerializer) } else if (classTag[T] == ClassTag.Double) { - rdd.setSerializer(classOf[DoubleVertexBroadcastMsgSerializer].getName) + rdd.setSerializer(new DoubleVertexBroadcastMsgSerializer) } rdd } @@ -104,11 +104,11 @@ object MsgRDDFunctions { // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { - rdd.setSerializer(classOf[IntAggMsgSerializer].getName) + rdd.setSerializer(new IntAggMsgSerializer) } else if (classTag[T] == ClassTag.Long) { - rdd.setSerializer(classOf[LongAggMsgSerializer].getName) + rdd.setSerializer(new LongAggMsgSerializer) } else if (classTag[T] == ClassTag.Double) { - rdd.setSerializer(classOf[DoubleAggMsgSerializer].getName) + rdd.setSerializer(new DoubleAggMsgSerializer) } rdd } 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 c74d487e206db..34a145e01818f 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 @@ -25,7 +25,7 @@ import org.apache.spark.graphx._ import org.apache.spark.serializer._ private[graphx] -class VertexIdMsgSerializer(conf: SparkConf) extends Serializer { +class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -46,7 +46,7 @@ class VertexIdMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for VertexBroadcastMessage[Int]. */ private[graphx] -class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { +class IntVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -70,7 +70,7 @@ class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for VertexBroadcastMessage[Long]. */ private[graphx] -class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { +class LongVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -94,7 +94,7 @@ class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for VertexBroadcastMessage[Double]. */ private[graphx] -class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { +class DoubleVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -118,7 +118,7 @@ class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for AggregationMessage[Int]. */ private[graphx] -class IntAggMsgSerializer(conf: SparkConf) extends Serializer { +class IntAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -142,7 +142,7 @@ class IntAggMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for AggregationMessage[Long]. */ private[graphx] -class LongAggMsgSerializer(conf: SparkConf) extends Serializer { +class LongAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -166,7 +166,7 @@ class LongAggMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for AggregationMessage[Double]. */ private[graphx] -class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer { +class DoubleAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { 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 e5a582b47ba05..73438d9535962 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala @@ -32,15 +32,14 @@ import org.apache.spark.serializer.SerializationStream class SerializerSuite extends FunSuite with LocalSparkContext { test("IntVertexBroadcastMsgSerializer") { - val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Int](3, 4, 5) val bout = new ByteArrayOutputStream - val outStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new IntVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new IntVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Int] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Int] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -54,15 +53,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongVertexBroadcastMsgSerializer") { - val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Long](3, 4, 5) val bout = new ByteArrayOutputStream - val outStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new LongVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new LongVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Long] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Long] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -76,15 +74,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleVertexBroadcastMsgSerializer") { - val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Double](3, 4, 5.0) val bout = new ByteArrayOutputStream - val outStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Double] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Double] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -98,15 +95,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("IntAggMsgSerializer") { - val conf = new SparkConf(false) val outMsg = (4: VertexId, 5) val bout = new ByteArrayOutputStream - val outStrm = new IntAggMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new IntAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntAggMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new IntAggMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: (VertexId, Int) = inStrm.readObject() val inMsg2: (VertexId, Int) = inStrm.readObject() assert(outMsg === inMsg1) @@ -118,15 +114,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongAggMsgSerializer") { - val conf = new SparkConf(false) val outMsg = (4: VertexId, 1L << 32) val bout = new ByteArrayOutputStream - val outStrm = new LongAggMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new LongAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongAggMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new LongAggMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: (VertexId, Long) = inStrm.readObject() val inMsg2: (VertexId, Long) = inStrm.readObject() assert(outMsg === inMsg1) @@ -138,15 +133,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleAggMsgSerializer") { - val conf = new SparkConf(false) val outMsg = (4: VertexId, 5.0) val bout = new ByteArrayOutputStream - val outStrm = new DoubleAggMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new DoubleAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleAggMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new DoubleAggMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: (VertexId, Double) = inStrm.readObject() val inMsg2: (VertexId, Double) = inStrm.readObject() assert(outMsg === inMsg1) From dc9654638f1d781ee1e54348fa41436b27793365 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sun, 16 Mar 2014 22:14:59 -0700 Subject: [PATCH 19/23] SPARK-1240: handle the case of empty RDD when takeSample https://spark-project.atlassian.net/browse/SPARK-1240 It seems that the current implementation does not handle the empty RDD case when run takeSample In this patch, before calling sample() inside takeSample API, I add a checker for this case and returns an empty Array when it's a empty RDD; also in sample(), I add a checker for the invalid fraction value In the test case, I also add several lines for this case Author: CodingCat Closes #135 from CodingCat/SPARK-1240 and squashes the following commits: fef57d4 [CodingCat] fix the same problem in PySpark 36db06b [CodingCat] create new test cases for takeSample from an empty red 810948d [CodingCat] further fix a40e8fb [CodingCat] replace if with require ad483fd [CodingCat] handle the case with empty RDD when take sample --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 7 ++++++- core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 7 +++++++ python/pyspark/rdd.py | 4 ++++ 3 files changed, 17 insertions(+), 1 deletion(-) 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 b50c9963b9d2c..f8283fbbb980d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -310,6 +310,7 @@ abstract class RDD[T: ClassTag]( * Return a sampled subset of this RDD. */ def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] = { + require(fraction >= 0.0, "Invalid fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) } else { @@ -344,6 +345,10 @@ abstract class RDD[T: ClassTag]( throw new IllegalArgumentException("Negative number of elements requested") } + if (initialCount == 0) { + return new Array[T](0) + } + if (initialCount > Integer.MAX_VALUE - 1) { maxSelected = Integer.MAX_VALUE - 1 } else { @@ -362,7 +367,7 @@ abstract class RDD[T: ClassTag]( var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() // If the first sample didn't turn out large enough, keep trying to take samples; - // this shouldn't happen often because we use a big multiplier for thei initial size + // this shouldn't happen often because we use a big multiplier for the initial size while (samples.length < total) { samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() } 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 60bcada55245b..9512e0e6eeb14 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -457,6 +457,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("takeSample") { val data = sc.parallelize(1 to 100, 2) + for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) assert(sample.size === 20) // Got exactly 20 elements @@ -488,6 +489,12 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } + test("takeSample from an empty rdd") { + val emptySet = sc.parallelize(Seq.empty[Int], 2) + val sample = emptySet.takeSample(false, 20, 1) + assert(sample.length === 0) + } + test("randomSplit") { val n = 600 val data = sc.parallelize(1 to n, 2) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6d549b40e5698..f3b432ff248a9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -268,6 +268,7 @@ def sample(self, withReplacement, fraction, seed): >>> sc.parallelize(range(0, 100)).sample(False, 0.1, 2).collect() #doctest: +SKIP [2, 3, 20, 21, 24, 41, 42, 66, 67, 89, 90, 98] """ + assert fraction >= 0.0, "Invalid fraction value: %s" % fraction return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) # this is ported from scala/spark/RDD.scala @@ -288,6 +289,9 @@ def takeSample(self, withReplacement, num, seed): if (num < 0): raise ValueError + if (initialCount == 0): + return list() + if initialCount > sys.maxint - 1: maxSelected = sys.maxint - 1 else: From 796977acdb5c96ca5c08591657137fb3e44d2e94 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Mar 2014 14:03:32 -0700 Subject: [PATCH 20/23] SPARK-1244: Throw exception if map output status exceeds frame size This is a very small change on top of @andrewor14's patch in #147. Author: Patrick Wendell Author: Andrew Or Closes #152 from pwendell/akka-frame and squashes the following commits: e5fb3ff [Patrick Wendell] Reversing test order 393af4c [Patrick Wendell] Small improvement suggested by Andrew Or 8045103 [Patrick Wendell] Breaking out into two tests 2b4e085 [Patrick Wendell] Consolidate Executor use of akka frame size c9b6109 [Andrew Or] Simplify test + make access to akka frame size more modular 281d7c9 [Andrew Or] Throw exception on spark.akka.frameSize exceeded + Unit tests --- .../org/apache/spark/MapOutputTracker.scala | 19 +++++- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 6 +- .../org/apache/spark/util/AkkaUtils.scala | 9 ++- .../org/apache/spark/AkkaUtilsSuite.scala | 10 ++-- .../apache/spark/MapOutputTrackerSuite.scala | 58 +++++++++++++++++-- 6 files changed, 84 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 5968973132942..80cbf951cb70e 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -35,13 +35,28 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage -private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster) +private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) extends Actor with Logging { + val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) + def receive = { case GetMapOutputStatuses(shuffleId: Int) => val hostPort = sender.path.address.hostPort logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) - sender ! tracker.getSerializedMapOutputStatuses(shuffleId) + val mapOutputStatuses = tracker.getSerializedMapOutputStatuses(shuffleId) + val serializedSize = mapOutputStatuses.size + if (serializedSize > maxAkkaFrameSize) { + val msg = s"Map output statuses were $serializedSize bytes which " + + s"exceeds spark.akka.frameSize ($maxAkkaFrameSize bytes)." + + /* For SPARK-1244 we'll opt for just logging an error and then throwing an exception. + * Note that on exception the actor will just restart. A bigger refactoring (SPARK-1239) + * will ultimately remove this entire code path. */ + val exception = new SparkException(msg) + logError(msg, exception) + throw exception + } + sender ! mapOutputStatuses case StopMapOutputTracker => logInfo("MapOutputTrackerActor stopped!") diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index d035d909b7b25..774cbd6441a48 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -191,7 +191,7 @@ object SparkEnv extends Logging { } mapOutputTracker.trackerActor = registerOrLookup( "MapOutputTracker", - new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster])) + new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) val shuffleFetcher = instantiateClass[ShuffleFetcher]( "spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher") 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 e69f6f72d3275..2ea2ec29f59f5 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -29,7 +29,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} -import org.apache.spark.util.Utils +import org.apache.spark.util.{AkkaUtils, Utils} /** * Spark executor used with Mesos, YARN, and the standalone scheduler. @@ -120,9 +120,7 @@ private[spark] class Executor( // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. - private val akkaFrameSize = { - env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size") - } + private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) // Start worker thread pool val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") 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 a6c9a9aaba8eb..d0ff17db632c1 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -49,7 +49,7 @@ private[spark] object AkkaUtils extends Logging { val akkaTimeout = conf.getInt("spark.akka.timeout", 100) - val akkaFrameSize = conf.getInt("spark.akka.frameSize", 10) + val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" if (!akkaLogLifecycleEvents) { @@ -92,7 +92,7 @@ private[spark] object AkkaUtils extends Logging { |akka.remote.netty.tcp.port = $port |akka.remote.netty.tcp.tcp-nodelay = on |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s - |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB + |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize |akka.log-config-on-start = $logAkkaConfig @@ -121,4 +121,9 @@ private[spark] object AkkaUtils extends Logging { def lookupTimeout(conf: SparkConf): FiniteDuration = { Duration.create(conf.get("spark.akka.lookupTimeout", "30").toLong, "seconds") } + + /** Returns the configured max frame size for Akka messages in bytes. */ + def maxFrameSizeBytes(conf: SparkConf): Int = { + conf.getInt("spark.akka.frameSize", 10) * 1024 * 1024 + } } diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala index cd054c1f684ab..d2e303d81c4c8 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -45,12 +45,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "true") badconf.set("spark.authenticate.secret", "bad") - val securityManagerBad = new SecurityManager(badconf); + val securityManagerBad = new SecurityManager(badconf) assert(securityManagerBad.isAuthenticationEnabled() === true) @@ -84,7 +84,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") @@ -136,7 +136,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val goodconf = new SparkConf goodconf.set("spark.authenticate", "true") @@ -189,7 +189,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 8efa072a97911..a5bd72eb0a122 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark import scala.concurrent.Await import akka.actor._ +import akka.testkit.TestActorRef import org.scalatest.FunSuite import org.apache.spark.scheduler.MapStatus @@ -51,14 +52,16 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) + tracker.trackerActor = + actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) + tracker.trackerActor = + actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -77,7 +80,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) + tracker.trackerActor = + actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -100,11 +104,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = new SecurityManager(conf)) - System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext + + // Will be cleared by LocalSparkContext + System.setProperty("spark.driver.port", boundPort.toString) val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = new SecurityManager(conf)) @@ -126,7 +132,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000, 0), size1000))) + Seq((BlockManagerId("a", "hostA", 1000, 0), size1000))) masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0)) masterTracker.incrementEpoch() @@ -136,4 +142,44 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { // failure should be cached intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } } + + test("remote fetch below akka frame size") { + val newConf = new SparkConf + newConf.set("spark.akka.frameSize", "1") + newConf.set("spark.akka.askTimeout", "1") // Fail fast + + val masterTracker = new MapOutputTrackerMaster(conf) + val actorSystem = ActorSystem("test") + val actorRef = TestActorRef[MapOutputTrackerMasterActor]( + new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) + val masterActor = actorRef.underlyingActor + + // Frame size should be ~123B, and no exception should be thrown + masterTracker.registerShuffle(10, 1) + masterTracker.registerMapOutput(10, 0, new MapStatus( + BlockManagerId("88", "mph", 1000, 0), Array.fill[Byte](10)(0))) + masterActor.receive(GetMapOutputStatuses(10)) + } + + test("remote fetch exceeds akka frame size") { + val newConf = new SparkConf + newConf.set("spark.akka.frameSize", "1") + newConf.set("spark.akka.askTimeout", "1") // Fail fast + + val masterTracker = new MapOutputTrackerMaster(conf) + val actorSystem = ActorSystem("test") + val actorRef = TestActorRef[MapOutputTrackerMasterActor]( + new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) + val masterActor = actorRef.underlyingActor + + // Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should throw exception. + // Note that the size is hand-selected here because map output statuses are compressed before + // being sent. + masterTracker.registerShuffle(20, 100) + (0 until 100).foreach { i => + masterTracker.registerMapOutput(20, i, new MapStatus( + BlockManagerId("999", "mps", 1000, 0), Array.fill[Byte](4000000)(0))) + } + intercept[SparkException] { masterActor.receive(GetMapOutputStatuses(20)) } + } } From 087eedca32fd87bfe1629588091bd307d45e4a7c Mon Sep 17 00:00:00 2001 From: Diana Carroll Date: Mon, 17 Mar 2014 17:35:51 -0700 Subject: [PATCH 21/23] [Spark-1261] add instructions for running python examples to doc overview page Author: Diana Carroll Closes #162 from dianacarroll/SPARK-1261 and squashes the following commits: 14ac602 [Diana Carroll] typo in python example text 5121e3e [Diana Carroll] Add explanation of how to run Python examples to main doc overview page --- docs/index.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/index.md b/docs/index.md index c4f4d79edbc6c..23311101e1712 100644 --- a/docs/index.md +++ b/docs/index.md @@ -23,10 +23,12 @@ For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_BI # Running the Examples and Shell -Spark comes with several sample programs in the `examples` directory. -To run one of the samples, use `./bin/run-example ` in the top-level Spark directory +Spark comes with several sample programs. Scala and Java examples are in the `examples` directory, and Python examples are in `python/examples`. +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 ./python/examples/pi.py local`. + Each example prints usage help when run with no parameters. Note that all of the sample programs take a `` parameter specifying the cluster URL From e3681f26fae7e87321ac991f5a0fb7517415803a Mon Sep 17 00:00:00 2001 From: Dan McClary Date: Tue, 18 Mar 2014 00:45:47 -0700 Subject: [PATCH 22/23] Spark 1246 add min max to stat counter Here's the addition of min and max to statscounter.py and min and max methods to rdd.py. Author: Dan McClary Closes #144 from dwmclary/SPARK-1246-add-min-max-to-stat-counter and squashes the following commits: fd3fd4b [Dan McClary] fixed error, updated test 82cde0e [Dan McClary] flipped incorrectly assigned inf values in StatCounter 5d96799 [Dan McClary] added max and min to StatCounter repr for pyspark 21dd366 [Dan McClary] added max and min to StatCounter output, updated doc 1a97558 [Dan McClary] added max and min to StatCounter output, updated doc a5c13b0 [Dan McClary] Added min and max to Scala and Java RDD, added min and max to StatCounter ed67136 [Dan McClary] broke min/max out into separate transaction, added to rdd.py 1e7056d [Dan McClary] added underscore to getBucket 37a7dea [Dan McClary] cleaned up boundaries for histogram -- uses real min/max when buckets are derived 29981f2 [Dan McClary] fixed indentation on doctest comment eaf89d9 [Dan McClary] added correct doctest for histogram 4916016 [Dan McClary] added histogram method, added max and min to statscounter --- .../apache/spark/api/java/JavaRDDLike.scala | 20 +++++++++++++++ .../main/scala/org/apache/spark/rdd/RDD.scala | 12 +++++++++ .../org/apache/spark/util/StatCounter.scala | 18 +++++++++++-- .../org/apache/spark/PartitioningSuite.scala | 2 ++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 2 ++ python/pyspark/rdd.py | 19 ++++++++++++++ python/pyspark/statcounter.py | 25 ++++++++++++++++--- 7 files changed, 93 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 2ba4fb8c18b1d..05b89b985736d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -478,6 +478,26 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } + /** + * Returns the maximum element from this RDD as defined by the specified + * Comparator[T]. + * @params comp the comparator that defines ordering + * @return the maximum of the RDD + * */ + def max(comp: Comparator[T]): T = { + rdd.max()(Ordering.comparatorToOrdering(comp)) + } + + /** + * Returns the minimum element from this RDD as defined by the specified + * Comparator[T]. + * @params comp the comparator that defines ordering + * @return the minimum of the RDD + * */ + def min(comp: Comparator[T]): T = { + rdd.min()(Ordering.comparatorToOrdering(comp)) + } + /** * Returns the first K elements from this RDD using the * natural ordering for T while maintain the order. 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 f8283fbbb980d..ddb901246d360 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -956,6 +956,18 @@ abstract class RDD[T: ClassTag]( */ def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse) + /** + * Returns the max of this RDD as defined by the implicit Ordering[T]. + * @return the maximum element of the RDD + * */ + def max()(implicit ord: Ordering[T]):T = this.reduce(ord.max) + + /** + * Returns the min of this RDD as defined by the implicit Ordering[T]. + * @return the minimum element of the RDD + * */ + def min()(implicit ord: Ordering[T]):T = this.reduce(ord.min) + /** * Save this RDD as a text file, using string representations of elements. */ diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala index f837dc7ccc860..732748a7ff82b 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -29,6 +29,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { private var n: Long = 0 // Running count of our values private var mu: Double = 0 // Running mean of our values private var m2: Double = 0 // Running variance numerator (sum of (x - mean)^2) + private var maxValue: Double = Double.NegativeInfinity // Running max of our values + private var minValue: Double = Double.PositiveInfinity // Running min of our values merge(values) @@ -41,6 +43,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { n += 1 mu += delta / n m2 += delta * (value - mu) + maxValue = math.max(maxValue, value) + minValue = math.min(minValue, value) this } @@ -58,7 +62,9 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { if (n == 0) { mu = other.mu m2 = other.m2 - n = other.n + n = other.n + maxValue = other.maxValue + minValue = other.minValue } else if (other.n != 0) { val delta = other.mu - mu if (other.n * 10 < n) { @@ -70,6 +76,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { } m2 += other.m2 + (delta * delta * n * other.n) / (n + other.n) n += other.n + maxValue = math.max(maxValue, other.maxValue) + minValue = math.min(minValue, other.minValue) } this } @@ -81,6 +89,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { other.n = n other.mu = mu other.m2 = m2 + other.maxValue = maxValue + other.minValue = minValue other } @@ -90,6 +100,10 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { def sum: Double = n * mu + def max: Double = maxValue + + def min: Double = minValue + /** Return the variance of the values. */ def variance: Double = { if (n == 0) { @@ -121,7 +135,7 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { def sampleStdev: Double = math.sqrt(sampleVariance) override def toString: String = { - "(count: %d, mean: %f, stdev: %f)".format(count, mean, stdev) + "(count: %d, mean: %f, stdev: %f, max: %f, min: %f)".format(count, mean, stdev, max, min) } } diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 4305686d3a6d5..996db70809320 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -171,6 +171,8 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(abs(6.0/2 - rdd.mean) < 0.01) assert(abs(1.0 - rdd.variance) < 0.01) assert(abs(1.0 - rdd.stdev) < 0.01) + assert(stats.max === 4.0) + assert(stats.min === 2.0) // Add other tests here for classes that should be able to handle empty partitions correctly } 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 9512e0e6eeb14..d6b5fdc7984b4 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -47,6 +47,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4))) assert(nums.collect({ case i if i >= 3 => i.toString }).collect().toList === List("3", "4")) assert(nums.keyBy(_.toString).collect().toList === List(("1", 1), ("2", 2), ("3", 3), ("4", 4))) + assert(nums.max() === 4) + assert(nums.min() === 1) val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) assert(partitionSums.collect().toList === List(3, 7)) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index f3b432ff248a9..ae09dbff02a36 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -571,7 +571,26 @@ def func(iterator): return reduce(op, vals, zeroValue) # TODO: aggregate + + + def max(self): + """ + Find the maximum item in this RDD. + + >>> sc.parallelize([1.0, 5.0, 43.0, 10.0]).max() + 43.0 + """ + return self.reduce(max) + def min(self): + """ + Find the maximum item in this RDD. + + >>> sc.parallelize([1.0, 5.0, 43.0, 10.0]).min() + 1.0 + """ + return self.reduce(min) + def sum(self): """ Add up the elements in this RDD. diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py index 8e1cbd4ad9856..080325061a697 100644 --- a/python/pyspark/statcounter.py +++ b/python/pyspark/statcounter.py @@ -26,7 +26,9 @@ def __init__(self, values=[]): self.n = 0L # Running count of our values self.mu = 0.0 # Running mean of our values self.m2 = 0.0 # Running variance numerator (sum of (x - mean)^2) - + self.maxValue = float("-inf") + self.minValue = float("inf") + for v in values: self.merge(v) @@ -36,6 +38,11 @@ def merge(self, value): self.n += 1 self.mu += delta / self.n self.m2 += delta * (value - self.mu) + if self.maxValue < value: + self.maxValue = value + if self.minValue > value: + self.minValue = value + return self # Merge another StatCounter into this one, adding up the internal statistics. @@ -49,7 +56,10 @@ def mergeStats(self, other): if self.n == 0: self.mu = other.mu self.m2 = other.m2 - self.n = other.n + self.n = other.n + self.maxValue = other.maxValue + self.minValue = other.minValue + elif other.n != 0: delta = other.mu - self.mu if other.n * 10 < self.n: @@ -58,6 +68,9 @@ def mergeStats(self, other): self.mu = other.mu - (delta * self.n) / (self.n + other.n) else: self.mu = (self.mu * self.n + other.mu * other.n) / (self.n + other.n) + + self.maxValue = max(self.maxValue, other.maxValue) + self.minValue = min(self.minValue, other.minValue) self.m2 += other.m2 + (delta * delta * self.n * other.n) / (self.n + other.n) self.n += other.n @@ -76,6 +89,12 @@ def mean(self): def sum(self): return self.n * self.mu + def min(self): + return self.minValue + + def max(self): + return self.maxValue + # Return the variance of the values. def variance(self): if self.n == 0: @@ -105,5 +124,5 @@ def sampleStdev(self): return math.sqrt(self.sampleVariance()) def __repr__(self): - return "(count: %s, mean: %s, stdev: %s)" % (self.count(), self.mean(), self.stdev()) + return "(count: %s, mean: %s, stdev: %s, max: %s, min: %s)" % (self.count(), self.mean(), self.stdev(), self.max(), self.min()) From e7423d4040ebd1ec4105d8d4b9a4a6600b18c2ac Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Mar 2014 00:46:03 -0700 Subject: [PATCH 23/23] Revert "SPARK-1236 - Upgrade Jetty to 9.1.3.v20140225." This reverts commit ca4bf8c572c2f70b484830f1db414b5073744ab6. Jetty 9 requires JDK7 which is probably not a dependency we want to bump right now. Before Spark 1.0 we should consider upgrading to Jetty 8. However, in the mean time to ease some pain let's revert this. Sorry for not catching this during the initial review. cc/ @rxin Author: Patrick Wendell Closes #167 from pwendell/jetty-revert and squashes the following commits: 811b1c5 [Patrick Wendell] Revert "SPARK-1236 - Upgrade Jetty to 9.1.3.v20140225." --- .../scala/org/apache/spark/HttpServer.scala | 21 +++---- .../org/apache/spark/ui/JettyUtils.scala | 58 +++++++++---------- pom.xml | 8 +-- project/SparkBuild.scala | 8 +-- 4 files changed, 46 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index d14693cb78a2d..cb5df25fa48df 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -21,9 +21,10 @@ import java.io.File import org.eclipse.jetty.util.security.{Constraint, Password} import org.eclipse.jetty.security.authentication.DigestAuthenticator -import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService} +import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService, SecurityHandler} -import org.eclipse.jetty.server.{Server, ServerConnector} +import org.eclipse.jetty.server.Server +import org.eclipse.jetty.server.bio.SocketConnector import org.eclipse.jetty.server.handler.{DefaultHandler, HandlerList, ResourceHandler} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -42,7 +43,7 @@ private[spark] class ServerStateException(message: String) extends Exception(mes */ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityManager) extends Logging { - private var server: Server = _ + private var server: Server = null private var port: Int = -1 def start() { @@ -50,16 +51,16 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan throw new ServerStateException("Server is already started") } else { logInfo("Starting HTTP Server") - val threadPool = new QueuedThreadPool - threadPool.setDaemon(true) - - server = new Server(threadPool) - val connector = new ServerConnector(server) - connector.setIdleTimeout(60 * 1000) + server = new Server() + val connector = new SocketConnector + connector.setMaxIdleTime(60*1000) connector.setSoLingerTime(-1) connector.setPort(0) server.addConnector(connector) + val threadPool = new QueuedThreadPool + threadPool.setDaemon(true) + server.setThreadPool(threadPool) val resHandler = new ResourceHandler resHandler.setResourceBase(resourceBase.getAbsolutePath) @@ -78,7 +79,7 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan } server.start() - port = connector.getLocalPort + port = server.getConnectors()(0).getLocalPort() } } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index cc7324939668b..7c35cd165ad7c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -19,7 +19,6 @@ package org.apache.spark.ui import java.net.InetSocketAddress import java.net.URL -import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletResponse, HttpServletRequest} import scala.annotation.tailrec @@ -29,7 +28,7 @@ import scala.xml.Node import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} -import org.eclipse.jetty.server.{NetworkConnector, Server} +import org.eclipse.jetty.server.{DispatcherType, Server} import org.eclipse.jetty.server.handler.HandlerList import org.eclipse.jetty.servlet.{DefaultServlet, FilterHolder, ServletContextHandler, ServletHolder} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -61,7 +60,8 @@ private[spark] object JettyUtils extends Logging { def createServlet[T <% AnyRef](servletParams: ServletParams[T], securityMgr: SecurityManager): HttpServlet = { new HttpServlet { - override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + override def doGet(request: HttpServletRequest, + response: HttpServletResponse) { if (securityMgr.checkUIViewPermissions(request.getRemoteUser())) { response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) response.setStatus(HttpServletResponse.SC_OK) @@ -72,7 +72,7 @@ private[spark] object JettyUtils extends Logging { response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.sendError(HttpServletResponse.SC_UNAUTHORIZED, - "User is not authorized to access this page.") + "User is not authorized to access this page."); } } } @@ -120,25 +120,26 @@ private[spark] object JettyUtils extends Logging { private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) - filters.foreach { filter => - if (!filter.isEmpty) { - logInfo("Adding filter: " + filter) - val holder : FilterHolder = new FilterHolder() - holder.setClassName(filter) - // get any parameters for each filter - val paramName = "spark." + filter + ".params" - val params = conf.get(paramName, "").split(',').map(_.trim()).toSet - params.foreach { - case param : String => - if (!param.isEmpty) { - val parts = param.split("=") - if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) - } + filters.foreach { + case filter : String => + if (!filter.isEmpty) { + logInfo("Adding filter: " + filter) + val holder : FilterHolder = new FilterHolder() + holder.setClassName(filter) + // get any parameters for each filter + val paramName = "spark." + filter + ".params" + val params = conf.get(paramName, "").split(',').map(_.trim()).toSet + params.foreach { + case param : String => + if (!param.isEmpty) { + val parts = param.split("=") + if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) + } + } + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, + DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) + handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } } - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, - DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) - handlers.foreach { handler => handler.addFilter(holder, "/*", enumDispatcher) } - } } } @@ -149,10 +150,7 @@ private[spark] object JettyUtils extends Logging { * If the desired port number is contented, continues incrementing ports until a free port is * found. Returns the chosen port and the jetty Server object. */ - def startJettyServer( - hostName: String, - port: Int, - handlers: Seq[ServletContextHandler], + def startJettyServer(hostName: String, port: Int, handlers: Seq[ServletContextHandler], conf: SparkConf): (Server, Int) = { addFilters(handlers, conf) @@ -162,18 +160,16 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { val server = new Server(new InetSocketAddress(hostName, currentPort)) - // Unfortunately Jetty 9 doesn't allow us to set both the thread pool and the port number in - // constructor. But fortunately the pool allocated by Jetty is always a QueuedThreadPool. - val pool = server.getThreadPool.asInstanceOf[QueuedThreadPool] + val pool = new QueuedThreadPool pool.setDaemon(true) - + server.setThreadPool(pool) server.setHandler(handlerList) Try { server.start() } match { case s: Success[_] => - (server, server.getConnectors.head.asInstanceOf[NetworkConnector].getLocalPort) + (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) diff --git a/pom.xml b/pom.xml index 377b6463136dd..524e5daff5388 100644 --- a/pom.xml +++ b/pom.xml @@ -186,22 +186,22 @@ org.eclipse.jetty jetty-util - 9.1.3.v20140225 + 7.6.8.v20121106 org.eclipse.jetty jetty-security - 9.1.3.v20140225 + 7.6.8.v20121106 org.eclipse.jetty jetty-plus - 9.1.3.v20140225 + 7.6.8.v20121106 org.eclipse.jetty jetty-server - 9.1.3.v20140225 + 7.6.8.v20121106 com.google.guava diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index db24f124f3cf8..aff191c98b409 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -238,10 +238,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % "9.1.3.v20140225", - "org.eclipse.jetty" % "jetty-util" % "9.1.3.v20140225", - "org.eclipse.jetty" % "jetty-plus" % "9.1.3.v20140225", - "org.eclipse.jetty" % "jetty-security" % "9.1.3.v20140225", + "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-util" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-plus" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-security" % "7.6.8.v20121106", /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"), "org.scalatest" %% "scalatest" % "1.9.1" % "test",